From 1c02a68cd87bbad1639a094b35557f8294069276 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 15 Feb 2024 09:37:25 +0300 Subject: [PATCH 01/85] Draft PR is moved to new repo --- .../combine_partial_final_agg.rs | 6 +- .../enforce_distribution.rs | 6 +- .../limited_distinct_aggregation.rs | 4 +- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../optimize_projections.rs | 5039 +++++++++++++++++ .../core/src/physical_optimizer/optimizer.rs | 13 +- .../physical_optimizer/projection_pushdown.rs | 2446 -------- .../physical_optimizer/topk_aggregation.rs | 2 +- datafusion/core/tests/sql/explain_analyze.rs | 22 +- datafusion/physical-expr/src/aggregate/mod.rs | 9 + datafusion/physical-expr/src/aggregate/sum.rs | 13 + .../physical-expr/src/window/window_expr.rs | 13 +- .../physical-plan/src/aggregates/mod.rs | 6 +- datafusion/physical-plan/src/lib.rs | 1 + datafusion/physical-plan/src/projection.rs | 28 +- datafusion/sqllogictest/test_files/cte.slt | 3 +- .../sqllogictest/test_files/distinct_on.slt | 2 +- .../sqllogictest/test_files/explain.slt | 10 +- .../join_disable_repartition_joins.slt | 17 +- datafusion/sqllogictest/test_files/limit.slt | 6 +- .../sqllogictest/test_files/predicates.slt | 16 +- datafusion/sqllogictest/test_files/select.slt | 1 + datafusion/sqllogictest/test_files/window.slt | 32 +- 23 files changed, 5170 insertions(+), 2527 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/optimize_projections.rs delete mode 100644 datafusion/core/src/physical_optimizer/projection_pushdown.rs diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 61eb2381c63b..93219246830a 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -70,12 +70,12 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { AggregateMode::Partial ) && can_combine( ( - agg_exec.group_by(), + agg_exec.group_expr(), agg_exec.aggr_expr(), agg_exec.filter_expr(), ), ( - input_agg_exec.group_by(), + input_agg_exec.group_expr(), input_agg_exec.aggr_expr(), input_agg_exec.filter_expr(), ), @@ -88,7 +88,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { }; AggregateExec::try_new( mode, - input_agg_exec.group_by().clone(), + input_agg_exec.group_expr().clone(), input_agg_exec.aggr_expr().to_vec(), input_agg_exec.filter_expr().to_vec(), input_agg_exec.input().clone(), diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 4f8806a68592..80418e0edbe7 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -460,7 +460,7 @@ fn reorder_aggregate_keys( ) -> Result { let parent_required = &agg_node.data; let output_columns = agg_exec - .group_by() + .group_expr() .expr() .iter() .enumerate() @@ -473,7 +473,7 @@ fn reorder_aggregate_keys( .collect::>(); if parent_required.len() == output_exprs.len() - && agg_exec.group_by().null_expr().is_empty() + && agg_exec.group_expr().null_expr().is_empty() && !physical_exprs_equal(&output_exprs, parent_required) { if let Some(positions) = expected_expr_positions(&output_exprs, parent_required) { @@ -481,7 +481,7 @@ fn reorder_aggregate_keys( agg_exec.input().as_any().downcast_ref::() { if matches!(agg_exec.mode(), &AggregateMode::Partial) { - let group_exprs = agg_exec.group_by().expr(); + let group_exprs = agg_exec.group_expr().expr(); let new_group_exprs = positions .into_iter() .map(|idx| group_exprs[idx].clone()) diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 9855247151b8..58d2ba84eeb9 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -52,7 +52,7 @@ impl LimitedDistinctAggregation { // We found what we want: clone, copy the limit down, and return modified node let new_aggr = AggregateExec::try_new( *aggr.mode(), - aggr.group_by().clone(), + aggr.group_expr().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), aggr.input().clone(), @@ -113,7 +113,7 @@ impl LimitedDistinctAggregation { if let Some(parent_aggr) = match_aggr.as_any().downcast_ref::() { - if !parent_aggr.group_by().eq(aggr.group_by()) { + if !parent_aggr.group_expr().eq(aggr.group_expr()) { // a partial and final aggregation with different groupings disqualifies // rewriting the child aggregation rewrite_applicable = false; diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index e990fead610d..c9931b3f06f8 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -28,10 +28,10 @@ pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; pub mod limited_distinct_aggregation; +mod optimize_projections; pub mod optimizer; pub mod output_requirements; pub mod pipeline_checker; -mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; mod sort_pushdown; diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs new file mode 100644 index 000000000000..5363032aa954 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -0,0 +1,5039 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! OptimizeProjections rule aims achieving the most effective use of projections +//! in plans. It ensures that query plans are free from unnecessary projections +//! and that no unused columns are propagated unnecessarily between plans. +//! +//! The rule is designed to enhance query performance by: +//! 1. Preventing the transfer of unused columns from leaves to root. +//! 2. Ensuring projections are used only when they contribute to narrowing the schema, +//! or when necessary for evaluation or aliasing. +//! +//! The optimization is conducted in two phases: +//! +//! Top-down Phase: +//! --------------- +//! - Traverses the plan from root to leaves. If the node is: +//! 1. Projection node, it may: +//! a) Merge it with its input projection if merge is beneficial. +//! b) Remove the projection if it is redundant. +//! c) Narrow the Projection if possible. +//! d) The projection can be nested into the source. +//! e) Do nothing, otherwise. +//! 2. Non-Projection node: +//! a) Schema needs pruning. Insert the necessary projections to the children. +//! b) All fields are required. Do nothing. +//! +//! Bottom-up Phase (now resides in map_children() implementation): +//! ---------------- +//! This pass is required because modifying a plan node can change the column +//! indices used by output nodes. When such a change occurs, we store the old +//! and new indices of the columns in the node's state. We then proceed from +//! the leaves to the root, updating the indices of columns in the plans by +//! referencing these mapping records. After the top-down phase, also some +//! unnecessary projections may emerge. When projections check its input schema +//! mapping, it can remove itself and assign new schema mapping to the new node +//! which was the projection's input formerly. + +use std::collections::{HashMap, HashSet}; +use std::mem; +use std::sync::Arc; + +use super::PhysicalOptimizerRule; +use crate::datasource::physical_plan::CsvExec; +use crate::error::Result; +use crate::physical_plan::filter::FilterExec; +use crate::physical_plan::projection::ProjectionExec; +use crate::physical_plan::ExecutionPlan; + +use arrow_schema::SchemaRef; +use chrono::naive; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::DataFusionError; +use datafusion_common::{internal_err, JoinSide, JoinType}; +use datafusion_physical_expr::expressions::{Column, Literal}; +use datafusion_physical_expr::utils::collect_columns; +use datafusion_physical_expr::{Partitioning, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::insert::FileSinkExec; +use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; +use datafusion_physical_plan::joins::{ + CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, + SymmetricHashJoinExec, +}; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::union::{InterleaveExec, UnionExec}; +use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion_physical_plan::{displayable, get_plan_string}; +use itertools::{Interleave, Itertools}; + +/// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary +/// fields for column requirements and changed indices of columns. +#[derive(Debug, Clone)] +pub struct ProjectionOptimizer { + pub plan: Arc, + /// The node above expects it can reach these columns. + /// Note: This set can be built on column indices rather than column expressions. + pub required_columns: HashSet, + /// The nodes above will be updated according to these mathces. First element indicates + /// the initial column index, and the second element is for the updated version. + pub schema_mapping: HashMap, + pub children_nodes: Vec, +} + +/// This type defines whether a column is required, in case of pairing with `true` value, or is +/// not required, in case of pairing with `false`. It is constructed based on output schema of a plan. +type ColumnRequirements = HashMap; + +impl ProjectionOptimizer { + /// Constructs the empty graph according to the plan. All state information is empty initially. + fn new_default(plan: Arc) -> Self { + let children = plan.children(); + Self { + plan, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: children.into_iter().map(Self::new_default).collect(), + } + } + + /// Recursively called transform function while traversing from root node + /// to leaf nodes. It only addresses the self and child node, and make + /// the necessary changes on them, does not deep dive. + fn adjust_node_with_requirements(mut self) -> Result { + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + + // If the node is a source provdider, no need a change. + if self.children_nodes.len() == 0 { + return Ok(self); + } + + if self.plan.as_any().is::() { + // If the node is a projection, it is analyzed and may be rewritten + // in a most effective way, or even removed. + self.optimize_projections() + } else { + // If the node corresponds to any other plan, a projection may be inserted to its input. + self.try_projection_insertion() + } + } + + /// The function tries 4 cases: + /// 1) If the input plan is also a projection, they can be merged into one projection. + /// 2) The projection can be removed. + /// 3) The projection can get narrower. + /// 4) The projection can be embedded into the source. + /// If none of them is possible, it remains unchanged. + pub fn optimize_projections(mut self) -> Result { + let projection_input = self.plan.children(); + let projection_input = projection_input[0].as_any(); + + // We first need to check having 2 sequential projections in case of merging them. + if projection_input.is::() { + self = match self.try_unifying_projections()? { + Transformed::Yes(unified_plans) => { + // We need to re-run the rule on the new node since it may need further optimizations. + // There may be 3 sequential projections, or the unified node may also be removed or narrowed. + return unified_plans.optimize_projections(); + } + Transformed::No(no_change) => no_change, + }; + } + + // The projection can be removed. To avoid making unnecessary operations, + // try_remove should be called before try_narrow. + self = match self.try_remove_projection() { + Transformed::Yes(removed) => { + // We need to re-run the rule on the current node. It is + // a new plan node and may need optimizations for sure. + return removed.adjust_node_with_requirements(); + } + Transformed::No(no_change) => no_change, + }; + + // The projection can get narrower. + self = match self.try_narrow_projection()? { + Transformed::Yes(narrowed) => { + return Ok(narrowed); + } + Transformed::No(no_change) => no_change, + }; + + // Source providers: + if projection_input.is::() { + self = match self.try_projected_csv() { + Transformed::Yes(new_csv) => return Ok(new_csv), + Transformed::No(no_change) => no_change, + } + } + + // If none of them possible, we will continue to next node. Output requirements + // of the projection in terms of projection input are inserted to child node. + let Some(projection_plan) = self.plan.as_any().downcast_ref::() + else { + return internal_err!( + "\"optimize_projections\" subrule must be used on ProjectionExec's." + ); + }; + // If there is nothing that could be better, insert the child requirements and continue. + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .flat_map(|e| collect_columns(&projection_plan.expr()[e.index()].0)) + .collect::>(); + Ok(self) + } + + /// Unifies `projection` with its input, which is also a [`ProjectionExec`], if it is beneficial. + fn try_unifying_projections(mut self) -> Result> { + // These are known to be a ProjectionExec. + let projection = self.plan.as_any().downcast_ref::().unwrap(); + let child_projection = self.children_nodes[0] + .plan + .as_any() + .downcast_ref::() + .unwrap(); + + if caching_projections(projection, child_projection) { + return Ok(Transformed::No(self)); + } + + let mut projected_exprs = vec![]; + for (expr, alias) in projection.expr() { + let Some(expr) = update_expr(expr, child_projection.expr(), true)? else { + return Ok(Transformed::No(self)); + }; + projected_exprs.push((expr, alias.clone())); + } + + let new_plan = + ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) + .map(|e| Arc::new(e) as _)?; + Ok(Transformed::Yes(ProjectionOptimizer { + plan: new_plan, + // Schema of the projection does not change, + // so no need any update on state variables. + required_columns: self.required_columns, + schema_mapping: self.schema_mapping, + children_nodes: self.children_nodes.swap_remove(0).children_nodes, + })) + } + + /// Tries to remove the [`ProjectionExec`]. When these conditions are satisfied, + /// the projection can be safely removed: + /// 1) Projection must have all column expressions without aliases. + /// 2) Projection input is fully required by the projection output requirements. + fn try_remove_projection(mut self) -> Transformed { + // It must be a projection + let projection_exec = + self.plan.as_any().downcast_ref::().unwrap(); + + // The projection must have all column expressions without aliases. + if !all_alias_free_columns(projection_exec.expr()) { + return Transformed::No(self); + } + // The expressions are known to be all columns. + let projection_columns = projection_exec + .expr() + .iter() + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) + .cloned() + .collect::>(); + + // Input requirements of the projection in terms of projection's parent requirements: + let projection_requires = self + .required_columns + .iter() + .map(|column| projection_columns[column.index()].clone()) + .collect::>(); + + // If all fields of the input are necessary, we can remove the projection. + let input_columns = collect_columns_in_plan_schema(projection_exec.input()); + if input_columns + .iter() + .all(|input_column| projection_requires.contains(&input_column)) + { + let new_mapping = self + .required_columns + .into_iter() + .filter_map(|column| { + let col_ind = column.index(); + if column != projection_columns[col_ind] { + Some((column, projection_columns[col_ind].clone())) + } else { + None + } + }) + .collect(); + + let replaced_child = self.children_nodes.swap_remove(0); + Transformed::Yes(ProjectionOptimizer { + plan: replaced_child.plan, + required_columns: projection_requires, + schema_mapping: new_mapping, + children_nodes: replaced_child.children_nodes, + }) + } else { + Transformed::No(self) + } + } + + /// Compares the inputs and outputs of the projection. If the projection can be + /// rewritten with a narrower schema, it is done so. Otherwise, it returns `None`. + fn try_narrow_projection(self) -> Result> { + // It must be a projection. + let projection_exec = + self.plan.as_any().downcast_ref::().unwrap(); + + // Check for the projection output if it has any redundant elements. + let projection_output_columns = projection_exec + .expr() + .iter() + .enumerate() + .map(|(i, (_e, a))| Column::new(a, i)) + .collect::>(); + let used_indices = projection_output_columns + .iter() + .filter(|&p_out| self.required_columns.contains(p_out)) + .map(|p_out| p_out.index()) + .collect::>(); + + if used_indices.len() == projection_output_columns.len() { + // All projected items are used. + return Ok(Transformed::No(self)); + } + + // New projected expressions are rewritten according to used indices. + let new_projection = used_indices + .iter() + .map(|i| projection_exec.expr()[*i].clone()) + .collect::>(); + + // Construct the mapping. + let mut schema_mapping = HashMap::new(); + for (new_idx, old_idx) in used_indices.iter().enumerate() { + if new_idx != *old_idx { + schema_mapping.insert( + projection_output_columns[*old_idx].clone(), + projection_output_columns[new_idx].clone(), + ); + } + } + + let new_projection_plan = Arc::new(ProjectionExec::try_new( + new_projection.clone(), + self.children_nodes[0].plan.clone(), + )?); + let new_projection_requires = self + .required_columns + .iter() + .map(|col| schema_mapping.get(col).cloned().unwrap_or(col.clone())) + .collect(); + let mut new_node = ProjectionOptimizer { + plan: new_projection_plan, + required_columns: new_projection_requires, + schema_mapping, + children_nodes: self.children_nodes, + }; + + // Since the rule work on the child node now, we need to insert child note requirements here. + new_node.children_nodes[0].required_columns = self + .required_columns + .iter() + .flat_map(|column| collect_columns(&new_projection[column.index()].0)) + .collect::>(); + + Ok(Transformed::Yes(new_node)) + } + + /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. + fn try_projected_csv(self) -> Transformed { + // These plans are known. + let projection = self.plan.as_any().downcast_ref::().unwrap(); + let csv = projection + .input() + .as_any() + .downcast_ref::() + .unwrap(); + // If there is any non-column or alias-carrier expression, Projection should not be removed. + // This process can be moved into CsvExec, but it could be a conflict of their responsibility. + if all_alias_free_columns(projection.expr()) { + let mut file_scan = csv.base_config().clone(); + let projection_columns = projection + .expr() + .iter() + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) + .collect::>(); + let new_projections = + new_projections_for_columns(&projection_columns, &file_scan.projection); + + file_scan.projection = Some(new_projections); + + Transformed::Yes(ProjectionOptimizer { + plan: Arc::new(CsvExec::new( + file_scan, + csv.has_header(), + csv.delimiter(), + csv.quote(), + csv.escape(), + csv.file_compression_type, + )) as _, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), // Sources cannot have a mapping. + children_nodes: vec![], + }) + } else { + Transformed::No(self) + } + } + + /// If the node plan can be rewritten with a narrower schema, a projection is inserted + /// into its input to do so. The node plans are rewritten according to its new input, + /// and the mapping of old indices vs. new indices is put to node's related field. + /// When this function returns and recursion on the node finishes, the upper node plans + /// are rewritten according to this mapping. This function also updates the parent + /// requirements and extends them with self requirements before inserting them to its child(ren). + fn try_projection_insertion(mut self) -> Result { + let plan = self.plan.clone(); + + if let Some(_projection) = plan.as_any().downcast_ref::() { + panic!( + "\"try_projection_insertion\" subrule cannot be used on ProjectionExec's." + ); + } else if let Some(_csv) = plan.as_any().downcast_ref::() { + panic!("\"try_projection_insertion\" subrule cannot be used on plans with no child.") + } + // These plans preserve the input schema, and do not add new requirements. + else if let Some(coal_b) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_coalesce_batches(coal_b)?; + } else if let Some(_) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_coalesce_partitions()?; + } else if let Some(glimit) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_global_limit(glimit)?; + } else if let Some(llimit) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_local_limit(llimit)?; + } + // These plans also preserve the input schema, but may extend requirements. + else if let Some(filter) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_filter(filter)?; + } else if let Some(repartition) = plan.as_any().downcast_ref::() + { + self = self.try_insert_below_repartition(repartition)?; + } else if let Some(sort) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_sort(sort)?; + } else if let Some(sortp_merge) = + plan.as_any().downcast_ref::() + { + self = self.try_insert_below_sort_preserving_merge(sortp_merge)?; + } + // Preserves schema and do not change requirements, but have multi-child. + else if let Some(_) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_union()?; + } else if let Some(_) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_interleave()?; + } + // Concatenates schemas and do not change requirements. + else if let Some(cj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_cross_join(cj)? + } + // Specially handled joins and aggregations + else if let Some(hj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_hash_join(hj)? + } else if let Some(nlj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_nested_loop_join(nlj)? + } else if let Some(smj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_sort_merge_join(smj)? + } else if let Some(shj) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_symmetric_hash_join(shj)? + } else if let Some(agg) = plan.as_any().downcast_ref::() { + if agg.aggr_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_none() + }) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self = self.try_insert_below_aggregate(agg)? + } else if let Some(w_agg) = plan.as_any().downcast_ref::() { + if w_agg.window_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_none() + }) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self = self.try_insert_below_window_aggregate(w_agg)? + } else if let Some(bw_agg) = plan.as_any().downcast_ref::() + { + if bw_agg.window_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_none() + }) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + self = self.try_insert_below_bounded_window_aggregate(bw_agg)? + } else if let Some(file_sink) = plan.as_any().downcast_ref::() { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan) + } else { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + return Ok(self); + } + Ok(self) + } + + fn try_insert_below_coalesce_batches( + mut self, + coal_batches: &CoalesceBatchesExec, + ) -> Result { + // CoalesceBatchesExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(CoalesceBatchesExec::new( + new_child.plan.clone(), + coal_batches.target_batch_size(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_coalesce_partitions(mut self) -> Result { + // CoalescePartitionsExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(CoalescePartitionsExec::new(new_child.plan.clone())) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_global_limit( + mut self, + glimit: &GlobalLimitExec, + ) -> Result { + // GlobalLimitExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if true { + // if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(GlobalLimitExec::new( + new_child.plan.clone(), + glimit.skip(), + glimit.fetch(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_local_limit( + mut self, + llimit: &LocalLimitExec, + ) -> Result { + // LocalLimitExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = + Arc::new(LocalLimitExec::new(new_child.plan.clone(), llimit.fetch())) + as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_filter( + mut self, + filter: &FilterExec, + ) -> Result { + // FilterExec extends the requirements with the columns in its predicate. + self.required_columns + .extend(collect_columns(filter.predicate())); + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the predicate with possibly updated column indices. + let new_predicate = update_column_index(filter.predicate(), &schema_mapping); + let plan = + Arc::new(FilterExec::try_new(new_predicate, new_child.plan.clone())?) + as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_repartition( + mut self, + repartition: &RepartitionExec, + ) -> Result { + // If RepartitionExec applies a hash repartition, it extends + // the requirements with the columns in the hashed expressions. + if let Partitioning::Hash(exprs, _size) = repartition.partitioning() { + self.required_columns + .extend(exprs.iter().flat_map(|expr| collect_columns(expr))); + } + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the hashed expressions if there is any with possibly updated column indices. + let new_partitioning = + if let Partitioning::Hash(exprs, size) = repartition.partitioning() { + Partitioning::Hash( + exprs + .iter() + .map(|expr| update_column_index(expr, &schema_mapping)) + .collect::>(), + *size, + ) + } else { + repartition.partitioning().clone() + }; + let plan = Arc::new(RepartitionExec::try_new( + new_child.plan.clone(), + new_partitioning, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_sort(mut self, sort: &SortExec) -> Result { + // SortExec extends the requirements with the columns in its sort expressions. + self.required_columns.extend( + sort.expr() + .iter() + .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), + ); + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_sort_exprs = sort + .expr() + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &schema_mapping), + options: sort_expr.options, + }) + .collect::>(); + let plan = Arc::new( + SortExec::new(new_sort_exprs, new_child.plan.clone()) + .with_preserve_partitioning(sort.preserve_partitioning()) + .with_fetch(sort.fetch()), + ) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_sort_preserving_merge( + mut self, + sortp_merge: &SortPreservingMergeExec, + ) -> Result { + // SortPreservingMergeExec extends the requirements with the columns in its sort expressions. + self.required_columns.extend( + sortp_merge + .expr() + .iter() + .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), + ); + + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns); + } else { + let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_sort_exprs = sortp_merge + .expr() + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &schema_mapping), + options: sort_expr.options, + }) + .collect::>(); + let plan = Arc::new( + SortPreservingMergeExec::new(new_sort_exprs, new_child.plan.clone()) + .with_fetch(sortp_merge.fetch()), + ) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: vec![new_child], + } + } + Ok(self) + } + + fn try_insert_below_union(mut self) -> Result { + // UnionExec does not change requirements. We can directly check whether there is a redundancy. + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + let required_columns = mem::take(&mut self.required_columns); + self.children_nodes + .iter_mut() + .for_each(|c| c.required_columns = required_columns.clone()); + } else { + let (new_children, schema_mapping) = + self.insert_multi_projection_below_union(requirement_map)?; + let plan = Arc::new(UnionExec::new( + new_children.iter().map(|c| c.plan.clone()).collect(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: new_children, + } + } + Ok(self) + } + + fn try_insert_below_interleave(mut self) -> Result { + let requirement_map = self.analyze_requirements(); + if all_columns_required(&requirement_map) { + let required_columns = mem::take(&mut self.required_columns); + self.children_nodes + .iter_mut() + .for_each(|c| c.required_columns = required_columns.clone()); + } else { + let (new_children, schema_mapping) = + self.insert_multi_projection_below_union(requirement_map)?; + let plan = Arc::new(InterleaveExec::try_new( + new_children.iter().map(|c| c.plan.clone()).collect(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), // clear the requirements + schema_mapping, + children_nodes: new_children, + } + } + Ok(self) + } + + fn try_insert_below_cross_join( + mut self, + cj: &CrossJoinExec, + ) -> Result { + let left_size = cj.left().schema().fields().len(); + // CrossJoinExec does not add new requirements. + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (true, true) => { + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(CrossJoinExec::new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + // Left child needs a projection. + (true, false) => { + let right_child = self.children_nodes.swap_remove(1); + let (new_left_child, left_schema_mapping) = + self.insert_projection_below_single_child(analyzed_join_left, 0)?; + let plan = Arc::new(CrossJoinExec::new( + new_left_child.plan.clone(), + right_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + // Right child needs a projection. + (false, true) => { + let left_child = self.children_nodes[0].clone(); + let (new_right_child, mut right_schema_mapping) = + self.insert_projection_below_single_child(analyzed_join_right, 1)?; + right_schema_mapping = right_schema_mapping + .into_iter() + .map(|(old, new)| { + ( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + left_size), + ) + }) + .collect(); + let plan = Arc::new(CrossJoinExec::new( + left_child.plan.clone(), + new_right_child.plan.clone(), + )) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (false, false) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + Ok(self) + } + + fn try_insert_below_hash_join( + mut self, + hj: &HashJoinExec, + ) -> Result { + let left_size = hj.left().schema().fields().len(); + // HashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. + match hj.join_type() { + JoinType::RightAnti | JoinType::RightSemi => { + self.required_columns = self + .required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() + } + _ => {} + } + self.required_columns + .extend(collect_columns_in_join_conditions( + hj.on(), + hj.filter(), + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match hj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_on = update_equivalence_conditions( + hj.on(), + &analyzed_join_left, + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(HashJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + hj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(HashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + hj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(HashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + hj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &analyzed_join_left, + &HashMap::new(), + ); + + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(HashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + hj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + hj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(HashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_nested_loop_join( + mut self, + nlj: &NestedLoopJoinExec, + ) -> Result { + let left_size = nlj.left().schema().fields().len(); + // NestedLoopJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. + self.required_columns + .extend(collect_columns_in_join_conditions( + &[], + nlj.filter(), + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match nlj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &analyzed_join_left, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_filter = update_non_equivalence_conditions( + nlj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(NestedLoopJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_filter, + nlj.join_type(), + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_sort_merge_join( + mut self, + smj: &SortMergeJoinExec, + ) -> Result { + let left_size = smj.left().schema().fields().len(); + // SortMergeJoin extends the requirements with the columns in its equivalence and non-equivalence conditions. + self.required_columns + .extend(collect_columns_in_join_conditions( + smj.on(), + None, + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match smj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + smj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &analyzed_join_left, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + smj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + smj.filter.as_ref(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SortMergeJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_symmetric_hash_join( + mut self, + shj: &SymmetricHashJoinExec, + ) -> Result { + let left_size = shj.left().schema().fields().len(); + // SymmetricHashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. + self.required_columns + .extend(collect_columns_in_join_conditions( + shj.on(), + shj.filter(), + left_size, + self.children_nodes[0].plan.schema(), + self.children_nodes[1].plan.schema(), + )); + let (analyzed_join_left, analyzed_join_right) = + self.analyze_requirements_of_joins(left_size); + + match shj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + match ( + all_columns_required(&analyzed_join_left), + all_columns_required(&analyzed_join_right), + ) { + // We need two projections on top of both children. + (false, false) => { + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_left, + &analyzed_join_right, + ); + let (new_left_child, new_right_child, schema_mapping) = self + .insert_multi_projections_below_join( + left_size, + analyzed_join_left, + analyzed_join_right, + )?; + + let plan = Arc::new(SymmetricHashJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + // TODO: update these + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping, + children_nodes: vec![new_left_child, new_right_child], + } + } + (false, true) => { + let right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_right, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + (true, false) => { + let left_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + // All columns are required. + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }) + } + } + } + JoinType::LeftAnti | JoinType::LeftSemi => { + match all_columns_required(&analyzed_join_left) { + false => { + let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( + shj.on(), + &analyzed_join_left, + &HashMap::new(), + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &analyzed_join_left, + &HashMap::new(), + ); + let (new_left_child, left_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_left, + 0, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + right_child.required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: left_schema_mapping, + children_nodes: vec![new_left_child, right_child], + } + } + true => { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes[1].required_columns = analyzed_join_right + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect() + } + } + } + JoinType::RightAnti | JoinType::RightSemi => { + match all_columns_required(&analyzed_join_right) { + false => { + let mut left_child = self.children_nodes.swap_remove(0); + let new_on = update_equivalence_conditions( + shj.on(), + &HashMap::new(), + &analyzed_join_right, + ); + let new_filter = update_non_equivalence_conditions( + shj.filter(), + &HashMap::new(), + &analyzed_join_right, + ); + let (new_right_child, right_schema_mapping) = self + .insert_projection_below_single_child( + analyzed_join_right, + 1, + )?; + let plan = Arc::new(SymmetricHashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + )?) as _; + + left_child.required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self = ProjectionOptimizer { + plan, + required_columns: HashSet::new(), + schema_mapping: right_schema_mapping, + children_nodes: vec![left_child, new_right_child], + } + } + true => { + self.children_nodes[0].required_columns = analyzed_join_left + .into_iter() + .filter_map( + |(column, used)| if used { Some(column) } else { None }, + ) + .collect(); + self.children_nodes[1].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[1].plan); + } + } + } + } + Ok(self) + } + + fn try_insert_below_aggregate( + mut self, + agg: &AggregateExec, + ) -> Result { + // `AggregateExec` applies their own projections. We can only limit + // the aggregate expressions unless they are used in the upper plans. + let group_columns_len = agg.group_expr().expr().len(); + let required_indices = self + .required_columns + .iter() + .map(|req_col| req_col.index()) + .collect::>(); + let unused_aggr_exprs = agg + .aggr_expr() + .iter() + .enumerate() + .filter(|(idx, _expr)| !required_indices.contains(&(idx + group_columns_len))) + .map(|(idx, _expr)| idx) + .collect::>(); + + if !unused_aggr_exprs.is_empty() { + let new_plan = AggregateExec::try_new( + agg.mode().clone(), + agg.group_expr().clone(), + agg.aggr_expr() + .iter() + .enumerate() + .filter(|(idx, _expr)| !unused_aggr_exprs.contains(idx)) + .map(|(_idx, expr)| expr.clone()) + .collect(), + agg.filter_expr().to_vec(), + agg.input().clone(), + agg.input_schema(), + )?; + self.children_nodes[0].required_columns = new_plan + .group_expr() + .expr() + .iter() + .map(|(e, alias)| collect_columns(e)) + .flatten() + .collect(); + self.children_nodes[0].required_columns.extend( + new_plan + .aggr_expr() + .iter() + .map(|e| { + e.expressions() + .iter() + .map(|e| collect_columns(e)) + .flatten() + .collect::>() + }) + .flatten(), + ); + self.plan = Arc::new(new_plan); + self.required_columns = HashSet::new(); + } else { + match agg.mode() { + datafusion_physical_plan::aggregates::AggregateMode::Final + | datafusion_physical_plan::aggregates::AggregateMode::FinalPartitioned => + { + let mut group_expr_len = agg.group_expr().expr().iter().count(); + let aggr_columns = agg + .aggr_expr() + .iter() + .flat_map(|e| { + e.state_fields() + .unwrap() + .iter() + .map(|field| { + group_expr_len += 1; + Column::new(field.name(), group_expr_len - 1) + }) + .collect::>() + }) + .collect::>(); + let group_columns = agg + .group_expr() + .expr() + .iter() + .flat_map(|(expr, _name)| collect_columns(expr)) + .collect::>(); + let filter_columns = agg + .filter_expr() + .iter() + .filter_map(|expr| expr.as_ref().map(collect_columns)) + .flatten() + .collect::>(); + self.children_nodes[0].required_columns.extend( + aggr_columns + .into_iter() + .chain(group_columns.into_iter()) + .chain(filter_columns.into_iter()), + ) + } + _ => { + let aggr_columns = agg + .aggr_expr() + .iter() + .flat_map(|e| { + e.expressions() + .iter() + .flat_map(collect_columns) + .collect::>() + }) + .collect::>(); + let group_columns = agg + .group_expr() + .expr() + .iter() + .flat_map(|(expr, _name)| collect_columns(expr)) + .collect::>(); + let filter_columns = agg + .filter_expr() + .iter() + .filter_map(|expr| expr.as_ref().map(collect_columns)) + .flatten() + .collect::>(); + self.children_nodes[0].required_columns.extend( + aggr_columns + .into_iter() + .chain(group_columns.into_iter()) + .chain(filter_columns.into_iter()), + ); + } + }; + } + Ok(self) + } + + fn try_insert_below_window_aggregate( + mut self, + w_agg: &WindowAggExec, + ) -> Result { + // Both tries to insert a projection to narrow input columns, and tries to narrow the window + // expressions. If none of them survives, we can even remove the window execution plan. + self.required_columns + .extend(w_agg.window_expr().iter().flat_map(|window_expr| { + window_expr + .expressions() + .iter() + .flat_map(|expr| collect_columns(&expr)) + .collect::>() + })); + self.required_columns.extend( + w_agg + .partition_keys + .iter() + .flat_map(|key| collect_columns(key)), + ); + let requirement_map = self.analyze_requirements(); + if !all_columns_required(&requirement_map) { + if window_agg_required( + w_agg.input().schema().fields().len(), + &requirement_map, + ) { + let (new_child, schema_mapping, window_usage) = self + .clone() + .insert_projection_below_window(w_agg, requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_window_exprs = w_agg + .window_expr() + .iter() + .zip(window_usage.clone()) + .filter(|(_window_expr, (_window_col, usage))| *usage) + .map(|(window_expr, (_window_col, _usage))| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, &schema_mapping)) + .collect(), + ) + }) + .collect::>>() + .unwrap(); + + let new_keys = w_agg + .partition_keys + .iter() + .zip(window_usage) + .filter_map(|(key, (_column, usage))| { + if usage { + Some(update_column_index(key, &schema_mapping)) + } else { + None + } + }) + .collect(); + let plan = Arc::new(WindowAggExec::try_new( + new_window_exprs, + new_child.plan.clone(), + new_keys, + )?) as _; + let required_columns = collect_columns_in_plan_schema(&plan); + self = ProjectionOptimizer { + plan, + required_columns, + schema_mapping, + children_nodes: vec![new_child], + } + } else { + // Remove the WindowAggExec + self = self.children_nodes.swap_remove(0); + self.required_columns = requirement_map + .into_iter() + .filter_map(|(column, used)| if used { Some(column) } else { None }) + .collect(); + } + } else { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < w_agg.schema().fields().len() - w_agg.window_expr().len() + }) + .cloned() + .collect(); + } + Ok(self) + } + + fn try_insert_below_bounded_window_aggregate( + mut self, + bw_agg: &BoundedWindowAggExec, + ) -> Result { + // Both tries to insert a projection to narrow input columns, and tries to narrow the window + // expressions. If none of them survives, we can even remove the window execution plan. + self.required_columns + .extend(bw_agg.window_expr().iter().flat_map(|window_expr| { + window_expr + .expressions() + .iter() + .flat_map(|expr| collect_columns(&expr)) + .collect::>() + })); + self.required_columns.extend( + bw_agg + .partition_keys + .iter() + .flat_map(|key| collect_columns(key)), + ); + let requirement_map = self.analyze_requirements(); + if !all_columns_required(&requirement_map) { + if window_agg_required( + bw_agg.input().schema().fields().len(), + &requirement_map, + ) { + let (new_child, schema_mapping, window_usage) = self + .clone() + .insert_projection_below_bounded_window(bw_agg, requirement_map)?; + // Rewrite the sort expressions with possibly updated column indices. + let new_window_exprs = bw_agg + .window_expr() + .iter() + .zip(window_usage.clone()) + .filter(|(_window_expr, (_window_col, usage))| *usage) + .map(|(window_expr, (_window_col, _usage))| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, &schema_mapping)) + .collect(), + ) + }) + .collect::>>() + .unwrap(); + + let new_keys = bw_agg + .partition_keys + .iter() + .zip(window_usage) + .filter_map(|(key, (_column, usage))| { + if usage { + Some(update_column_index(key, &schema_mapping)) + } else { + None + } + }) + .collect(); + let plan = Arc::new(BoundedWindowAggExec::try_new( + new_window_exprs, + new_child.plan.clone(), + new_keys, + bw_agg.input_order_mode.clone(), + )?) as _; + let required_columns = collect_columns_in_plan_schema(&plan); + self = ProjectionOptimizer { + plan, + required_columns, + schema_mapping, + children_nodes: vec![new_child], + } + } else { + // Remove the WindowAggExec + self = self.children_nodes.swap_remove(0); + self.required_columns = requirement_map + .into_iter() + .filter_map(|(column, used)| if used { Some(column) } else { None }) + .collect(); + } + } else { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < bw_agg.schema().fields().len() - bw_agg.window_expr().len() + }) + .cloned() + .collect(); + } + Ok(self) + } + + /// Compares the required and existing columns in the node, and maps them accordingly. Caller side must + /// ensure that the node extends its own requirements if the node's plan can introduce new requirements. + fn analyze_requirements(&self) -> ColumnRequirements { + let mut requirement_map = HashMap::new(); + let columns_in_schema = collect_columns_in_plan_schema(&self.plan); + columns_in_schema.into_iter().for_each(|col| { + let contains = self.required_columns.contains(&col); + requirement_map.insert(col, contains); + }); + requirement_map + } + + /// Compares the columns required from the left/right child and existing columns in the left/right + /// child. If there is any redundant field, it returns the mapping of columns whether it is required + /// or not. If there is no redundancy, it returns `None` for that child. Caller side must ensure + /// that the join node extends its own requirements if the node's plan can introduce new requirements. + /// Each column refers to its own table schema index, not to the join output schema. + fn analyze_requirements_of_joins( + &self, + left_size: usize, + ) -> (ColumnRequirements, ColumnRequirements) { + let columns_in_schema = + collect_columns_in_plan_schema(&self.children_nodes[0].plan) + .into_iter() + .chain( + collect_columns_in_plan_schema(&self.children_nodes[1].plan) + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)), + ); + let requirement_map = columns_in_schema + .into_iter() + .map(|col| { + if self.required_columns.contains(&col) { + (col, true) + } else { + (col, false) + } + }) + .collect::>(); + + let (requirement_map_left, mut requirement_map_right) = requirement_map + .into_iter() + .partition::, _>(|(col, _)| col.index() < left_size); + + requirement_map_right = requirement_map_right + .into_iter() + .map(|(col, used)| (Column::new(col.name(), col.index() - left_size), used)) + .collect::>(); + + (requirement_map_left, requirement_map_right) + } + + /// If a node is known to have redundant columns, we need to insert a projection to its input. + /// This function takes this node and requirement mapping of this node. Then, defines the projection + /// and constructs the new subtree. The returned objects are the new tree starting from the inserted + /// projection, and the mapping of columns referring to the schemas of pre-insertion and post-insertion. + fn insert_projection( + self, + requirement_map: ColumnRequirements, + ) -> Result<(Self, HashMap)> { + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let mut unused_columns = HashSet::new(); + let mut projected_exprs = requirement_map + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect::>(); + projected_exprs.sort_by_key(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap().index() + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter() { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + // Required columns must have been extended with self node requirements before this point. + required_columns: new_requirements, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes, + }; + Ok((inserted_projection, new_mapping)) + } + + /// Multi-child version of `insert_projection` for `UnionExec`'s. + fn insert_multi_projection_below_union( + self, + requirement_map: ColumnRequirements, + ) -> Result<(Vec, HashMap)> { + // During the iteration, we construct the ProjectionExec's with required columns as the new children, + // and also collect the unused columns to store the index changes after removal of some columns. + let mut unused_columns = HashSet::new(); + let mut projected_exprs = requirement_map + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect::>(); + projected_exprs.sort_by_key(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap().index() + }); + let inserted_projections = self + .plan + .children() + .into_iter() + .map(|child_plan| { + Ok(Arc::new(ProjectionExec::try_new( + projected_exprs.clone(), + child_plan, + )?) as _) + }) + .collect::>>()?; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter() { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = inserted_projections + .iter() + .map(|inserted_projection| { + collect_columns_in_plan_schema(inserted_projection) + }) + .collect::>(); + let inserted_projection_nodes = inserted_projections + .into_iter() + .zip(self.children_nodes) + .enumerate() + .map(|(idx, (p, child))| ProjectionOptimizer { + plan: p, + required_columns: new_requirements[idx].clone(), + schema_mapping: HashMap::new(), + children_nodes: vec![child], + }) + .collect(); + Ok((inserted_projection_nodes, new_mapping)) + } + + /// Single child version of `insert_projection` for joins. + fn insert_projection_below_single_child( + self, + requirement_map_left: ColumnRequirements, + children_index: usize, + ) -> Result<(Self, HashMap)> { + let mut unused_columns = HashSet::new(); + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let mut projected_exprs = requirement_map_left + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect::>(); + projected_exprs.sort_by_key(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap().index() + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs.clone(), + self.plan.children()[children_index].clone(), + )?) as _; + + let required_columns = projected_exprs + .iter() + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) + .collect::>(); + + let mut new_mapping = HashMap::new(); + for col in required_columns.into_iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter() { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes[children_index].clone()], + }; + Ok((inserted_projection, new_mapping)) + } + + /// Multi-child version of `insert_projection` for joins. + fn insert_multi_projections_below_join( + self, + left_size: usize, + requirement_map_left: ColumnRequirements, + requirement_map_right: ColumnRequirements, + ) -> Result<(Self, Self, HashMap)> { + let original_right = self.children_nodes[1].plan.clone(); + let (new_left_child, mut left_schema_mapping) = self + .clone() + .insert_projection_below_single_child(requirement_map_left, 0)?; + let (new_right_child, right_schema_mapping) = + self.insert_projection_below_single_child(requirement_map_right, 1)?; + + let new_left_size = new_left_child.plan.schema().fields().len(); + // left_schema_mapping does not need to be change, but it is updated with + // those coming form the right side to represent overall join output mapping. + for (idx, field) in + original_right + .schema() + .fields() + .iter() + .enumerate() + .filter(|(idx, field)| { + let right_projection = new_right_child + .plan + .as_any() + .downcast_ref::() + .unwrap() + .expr() + .iter() + .map(|(expr, _alias)| { + expr.as_any().downcast_ref::().unwrap() + }) + .collect::>(); + right_projection.contains(&&Column::new(field.name(), *idx)) + }) + { + left_schema_mapping.insert( + Column::new(field.name(), idx + left_size), + Column::new(field.name(), idx + new_left_size), + ); + } + for (old, new) in right_schema_mapping.into_iter() { + left_schema_mapping.insert( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + new_left_size), + ); + } + Ok((new_left_child, new_right_child, left_schema_mapping)) + } + + /// `insert_projection` for windows. + fn insert_projection_below_window( + self, + w_agg: &WindowAggExec, + requirement_map: ColumnRequirements, + ) -> Result<(Self, HashMap, ColumnRequirements)> { + let original_schema_len = w_agg.schema().fields().len(); + let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map + .into_iter() + .partition(|(column, _used)| column.index() < original_schema_len); + let mut unused_columns = HashSet::new(); + + let projected_exprs = base + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect(); + window.iter().for_each(|(col, used)| { + if !used { + unused_columns.insert(col.clone()); + } + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter().chain(unused_columns.iter()) { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + // Required columns must have been extended with self node requirements before this point. + required_columns: new_requirements, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes, + }; + Ok((inserted_projection, new_mapping, window)) + } + + /// `insert_projection` for bounded windows. + fn insert_projection_below_bounded_window( + self, + bw_agg: &BoundedWindowAggExec, + requirement_map: ColumnRequirements, + ) -> Result<(Self, HashMap, ColumnRequirements)> { + let original_schema_len = bw_agg.schema().fields().len(); + let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map + .into_iter() + .partition(|(column, _used)| column.index() < original_schema_len); + let mut unused_columns = HashSet::new(); + + let projected_exprs = base + .into_iter() + .filter_map(|(col, used)| { + if used { + let col_name = col.name().to_string(); + Some((Arc::new(col) as Arc, col_name)) + } else { + unused_columns.insert(col); + None + } + }) + .collect(); + window.iter().for_each(|(col, used)| { + if !used { + unused_columns.insert(col.clone()); + } + }); + let inserted_projection = Arc::new(ProjectionExec::try_new( + projected_exprs, + self.plan.children()[0].clone(), + )?) as _; + + let mut new_mapping = HashMap::new(); + for col in self.required_columns.iter() { + let mut skipped_columns = 0; + for unused_col in unused_columns.iter().chain(unused_columns.iter()) { + if unused_col.index() < col.index() { + skipped_columns += 1; + } + } + if skipped_columns > 0 { + new_mapping.insert( + col.clone(), + Column::new(col.name(), col.index() - skipped_columns), + ); + } + } + + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + // Required columns must have been extended with self node requirements before this point. + required_columns: new_requirements, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes, + }; + Ok((inserted_projection, new_mapping, window)) + } + + /// Responsible for updating the node's plan with new children and possibly updated column indices, + /// and for transferring the column mapping to the upper nodes. There is an exception for the + /// projection nodes; they may be removed also in case of being considered as unnecessary, + /// which leads to re-update the mapping after removal. + fn index_updater(mut self: ProjectionOptimizer) -> Result> { + let mut all_mappings = self + .children_nodes + .iter() + .map(|node| node.schema_mapping.clone()) + .collect::>(); + if !all_mappings.iter().all(|map| map.is_empty()) { + // The self plan will update its column indices according to the changes its children schemas. + let plan_copy = self.plan.clone(); + let plan_any = plan_copy.as_any(); + + // These plans do not have any expression related field. + // They simply transfer the mapping to the parent node. + if let Some(_coal_batches) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_coal_parts) = + plan_any.downcast_ref::() + { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_glimit) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_llimit) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_union) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_union) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(_cj) = plan_any.downcast_ref::() { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + self.update_mapping(all_mappings) + } else if let Some(projection) = plan_any.downcast_ref::() { + self.plan = rewrite_projection( + projection, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + // Rewriting the projection does not change its output schema, + // and projections does not need to transfer the mapping to upper nodes. + } else if let Some(filter) = plan_any.downcast_ref::() { + self.plan = rewrite_filter( + filter.predicate(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(repartition) = plan_any.downcast_ref::() { + self.plan = rewrite_repartition( + repartition.partitioning(), + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(sort) = plan_any.downcast_ref::() { + self.plan = rewrite_sort( + sort, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(sortp_merge) = + plan_any.downcast_ref::() + { + self.plan = rewrite_sort_preserving_merge( + sortp_merge, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )?; + self.update_mapping(all_mappings) + } else if let Some(hj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_hash_join( + hj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + match hj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => { + let (new_left, new_right) = + new_mapping.into_iter().partition(|(col_initial, _)| { + col_initial.index() < left_size + }); + all_mappings.push(new_left); + all_mappings.push(new_right); + } + JoinType::LeftSemi | JoinType::LeftAnti => { + all_mappings.push(left_mapping) + } + JoinType::RightAnti | JoinType::RightSemi => { + all_mappings.push(right_mapping) + } + }; + self.update_mapping(all_mappings) + } else if let Some(nlj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_nested_loop_join( + nlj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + all_mappings[0] = match nlj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => new_mapping, + JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, + JoinType::RightAnti | JoinType::RightSemi => right_mapping, + }; + self.update_mapping(all_mappings) + } else if let Some(smj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_sort_merge_join( + smj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + all_mappings[0] = match smj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => new_mapping, + JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, + JoinType::RightAnti | JoinType::RightSemi => right_mapping, + }; + self.update_mapping(all_mappings) + } else if let Some(shj) = plan_any.downcast_ref::() { + let left_size = self.children_nodes[0].plan.schema().fields().len(); + let left_mapping = all_mappings.swap_remove(0); + let right_mapping = all_mappings.swap_remove(0); + let new_mapping = left_mapping + .iter() + .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .chain(right_mapping.iter().map(|(initial, new)| { + ( + Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(new.name(), new.index() + left_size), + ) + })) + .collect::>(); + self.plan = rewrite_symmetric_hash_join( + shj, + self.children_nodes[0].plan.clone(), + self.children_nodes[1].plan.clone(), + &new_mapping, + left_size, + )?; + all_mappings[0] = match shj.join_type() { + JoinType::Right + | JoinType::Full + | JoinType::Left + | JoinType::Inner => new_mapping, + JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, + JoinType::RightAnti | JoinType::RightSemi => right_mapping, + }; + self.update_mapping(all_mappings) + } else if let Some(agg) = plan_any.downcast_ref::() { + self.plan = if let Some(updated) = rewrite_aggregate( + agg, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )? { + updated + } else { + return Ok(Transformed::No(self)); + }; + self.update_mapping(all_mappings) + } else if let Some(w_agg) = plan_any.downcast_ref::() { + self.plan = if let Some(updated) = rewrite_window_aggregate( + w_agg, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )? { + updated + } else { + return Ok(Transformed::No(self)); + }; + self.update_mapping(all_mappings) + } else if let Some(bw_agg) = plan_any.downcast_ref::() { + self.plan = if let Some(updated) = rewrite_bounded_window_aggregate( + bw_agg, + self.children_nodes[0].plan.clone(), + &all_mappings[0], + )? { + updated + } else { + return Ok(Transformed::No(self)); + }; + self.update_mapping(all_mappings) + } else if let Some(file_sink) = plan_any.downcast_ref::() { + let mapped_exprs = + all_mappings.swap_remove(0).into_iter().collect::>(); + let mut existing_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan) + .into_iter() + .collect_vec(); + existing_columns.sort_by_key(|col| col.index()); + let mut exprs = vec![]; + for idx in 0..existing_columns.len() { + if let Some((initial, _final)) = mapped_exprs + .iter() + .find(|(initial, _final)| initial.index() == idx) + { + exprs.push(( + Arc::new(initial.clone()) as Arc, + initial.name().to_string(), + )); + } else { + exprs.push(( + Arc::new(existing_columns[idx].clone()) + as Arc, + existing_columns[idx].name().to_string(), + )); + } + } + let projection = Arc::new(ProjectionExec::try_new( + exprs, + self.children_nodes[0].plan.clone(), + )?); + let new_child = ProjectionOptimizer { + plan: projection, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + self.plan = self.plan.with_new_children(vec![new_child.plan.clone()])?; + self.children_nodes = vec![new_child]; + } else { + unreachable!() + } + } else { + self.plan = self.plan.with_new_children( + self.children_nodes + .iter() + .map(|child| child.plan.clone()) + .collect(), + )?; + } + + Ok(Transformed::Yes(self)) + } + + fn update_mapping(&mut self, mut child_mappings: Vec>) { + if self.schema_mapping.is_empty() { + self.schema_mapping = child_mappings.swap_remove(0); + } else { + let child_map = child_mappings.swap_remove(0); + self.schema_mapping = self + .schema_mapping + .iter() + .map(|(initial, new)| { + ( + initial.clone(), + child_map.get(&new).cloned().unwrap_or(new.clone()), + ) + }) + .collect() + } + } + + /// After the top-down pass, there may be some unnecessary projections surviving + /// since they assumes themselves as necessary when they are analyzed, but after + /// some optimizations below, they may become unnecessary. This function checks + /// if the projection is still necessary. If it is not so, it is removed, and + /// a new mapping is set to the new node, which is the child of the projection, + /// to transfer the changes resulting from the removal of the projection. + fn try_remove_projection_bottom_up(mut self) -> Result { + let plan = self.plan.clone(); + let Some(projection) = plan.as_any().downcast_ref::() else { + return Ok(self); + }; + // Is the projection really required? First, we need to + // have all column expression in the projection for removal. + if all_alias_free_columns(projection.expr()) { + // Then, check if all columns in the input schema exist after + // the projection. If it is so, we can remove the projection + // since it does not provide any benefit. + let child_columns = collect_columns_in_plan_schema(projection.input()); + let projection_columns = projection + .expr() + .iter() + .map(|(expr, _alias)| { + // We have ensured all expressions are column. + expr.as_any().downcast_ref::().unwrap().clone() + }) + .collect::>(); + if child_columns + .iter() + .all(|child_col| projection_columns.contains(child_col)) + { + // We need to store the existing node's mapping. + let self_mapping = self.schema_mapping; + // Remove the projection node. + self = self.children_nodes.swap_remove(0); + + if self_mapping.is_empty() { + self.schema_mapping = projection + .expr() + .iter() + .enumerate() + .filter_map(|(idx, (col, _alias))| { + let new_column = + col.as_any().downcast_ref::().unwrap(); + if new_column.index() != idx { + Some(( + Column::new(new_column.name(), idx), + new_column.clone(), + )) + } else { + None + } + }) + .collect(); + } else { + self.schema_mapping = self_mapping + .into_iter() + .map(|(expected, updated)| { + ( + expected, + Column::new( + updated.name(), + projection_columns[updated.index()].index(), + ), + ) + }) + .collect() + } + } + } + return Ok(self); + } +} + +impl TreeNode for ProjectionOptimizer { + fn apply_children(&self, op: &mut F) -> Result + where + F: FnMut(&Self) -> Result, + { + for child in &self.children_nodes { + match op(child)? { + VisitRecursion::Continue => {} + VisitRecursion::Skip => return Ok(VisitRecursion::Continue), + VisitRecursion::Stop => return Ok(VisitRecursion::Stop), + } + } + Ok(VisitRecursion::Continue) + } + + fn map_children(mut self, transform: F) -> Result + where + F: FnMut(Self) -> Result, + { + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + + if self.children_nodes.is_empty() { + Ok(self) + } else { + self.children_nodes = self + .children_nodes + .into_iter() + .map(transform) + .collect::>>()?; + + self = match self.index_updater()? { + Transformed::Yes(updated) => updated, + Transformed::No(not_rewritable) => { + ProjectionOptimizer::new_default(not_rewritable.plan) + } + }; + // After the top-down pass, there may be some unnecessary projections surviving + // since they assumes themselves as necessary when they are analyzed, but after + // some optimizations below, they may become unnecessary. This check is done + // here, and if the projection is regarded as unnecessary, the removal would + // set a new the mapping on the new node, which is the child of the projection. + self = self.try_remove_projection_bottom_up()?; + Ok(self) + } + } +} + +#[derive(Default)] +pub struct OptimizeProjections {} + +impl OptimizeProjections { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} +fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) +} +impl PhysicalOptimizerRule for OptimizeProjections { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + // Collect initial columns requirements from the plan's schema. + let initial_requirements = collect_columns_in_plan_schema(&plan); + let mut optimizer = ProjectionOptimizer::new_default(plan); + // Insert the initial requirements to the root node, and run the rule. + optimizer.required_columns = initial_requirements.clone(); + let mut optimized = optimizer.transform_down(&|o| { + o.adjust_node_with_requirements().map(Transformed::Yes) + })?; + // Ensure the final optimized plan satisfies the initial schema requirements. + optimized = satisfy_initial_schema(optimized, initial_requirements)?; + + // TODO: Remove this check to tests + crosscheck_helper(optimized.clone())?; + + Ok(optimized.plan) + } + + fn name(&self) -> &str { + "OptimizeProjections" + } + + fn schema_check(&self) -> bool { + true + } +} + +// TODO: Remove this to tests +pub fn crosscheck_helper(context: ProjectionOptimizer) -> Result<()> { + context.transform_up(&|node| { + assert_eq!(node.children_nodes.len(), node.plan.children().len()); + if !node.children_nodes.is_empty() { + assert_eq!( + get_plan_string(&node.plan), + get_plan_string(&node.plan.clone().with_new_children( + node.children_nodes.iter().map(|c| c.plan.clone()).collect() + )?) + ); + } + Ok(Transformed::No(node)) + })?; + + Ok(()) +} + +/// Ensures that the output schema `po` matches the `initial_requirements`. +/// If the `schema_mapping` of `po` indicates that some columns have been re-mapped, +/// a new projection is added to restore the initial column order and indices. +fn satisfy_initial_schema( + po: ProjectionOptimizer, + initial_requirements: HashSet, +) -> Result { + if collect_columns_in_plan_schema(&po.plan) == initial_requirements { + // The initial schema is already satisfied, no further action required. + Ok(po) + } else { + // Collect expressions for the final projection to match the initial requirements. + let mut initial_requirements_vec = + initial_requirements.clone().into_iter().collect_vec(); + initial_requirements_vec.sort_by_key(|expr| expr.index()); + let projected_exprs = initial_requirements_vec + .iter() + .map(|col| { + // If there is a change, get the new index. + let column_index = po.schema_mapping.get(&col).unwrap_or(&col).index(); + let new_col = Arc::new(Column::new(col.name(), column_index)) + as Arc; + (new_col, col.name().to_string()) + }) + .collect::>(); + + // Create the final projection to align with the initial schema. + let final_projection = + Arc::new(ProjectionExec::try_new(projected_exprs, po.plan.clone())?); + + // Return a new ProjectionOptimizer with the final projection, resetting the schema mapping. + Ok(ProjectionOptimizer { + plan: final_projection, + required_columns: initial_requirements, + schema_mapping: HashMap::new(), // Reset schema mapping as we've now satisfied the initial schema + children_nodes: vec![po], // Keep the original node as the child + }) + } +} + +/// Iterates over all columns and returns true if all columns are required. +fn all_columns_required(requirement_map: &ColumnRequirements) -> bool { + requirement_map.iter().all(|(_k, v)| *v) +} + +fn window_agg_required( + original_schema_len: usize, + requirements: &ColumnRequirements, +) -> bool { + requirements + .iter() + .filter(|(column, _used)| column.index() >= original_schema_len) + .any(|(_column, used)| *used) +} + +// If an expression is not trivial and it is referred more than 1, +// unification will not be beneficial as going against caching mechanism +// for non-trivial computations. See the discussion: +// https://github.com/apache/arrow-datafusion/issues/8296 +fn caching_projections( + projection: &ProjectionExec, + child_projection: &ProjectionExec, +) -> bool { + let mut column_ref_map: HashMap = HashMap::new(); + // Collect the column references' usage in the parent projection. + projection.expr().iter().for_each(|(expr, _)| { + expr.apply(&mut |expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + VisitRecursion::Continue + }) + }) + .unwrap(); + }); + column_ref_map.iter().any(|(column, count)| { + *count > 1 && !is_expr_trivial(&child_projection.expr()[column.index()].0) + }) +} + +/// Checks if the given expression is trivial. +/// An expression is considered trivial if it is either a `Column` or a `Literal`. +fn is_expr_trivial(expr: &Arc) -> bool { + expr.as_any().downcast_ref::().is_some() + || expr.as_any().downcast_ref::().is_some() +} + +/// Given the expression set of a projection, checks if the projection causes +/// any renaming or constructs a non-`Column` physical expression. +fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { + exprs.iter().all(|(expr, alias)| { + expr.as_any() + .downcast_ref::() + .map(|column| column.name() == alias) + .unwrap_or(false) + }) +} + +/// Updates a source provider's projected columns according to the given +/// projection operator's expressions. To use this function safely, one must +/// ensure that all expressions are `Column` expressions without aliases. +fn new_projections_for_columns( + projection: &[&Column], + source: &Option>, +) -> Vec { + projection + .iter() + .filter_map(|col| source.as_ref().map(|proj| proj[col.index()])) + .collect() +} + +#[derive(Debug, PartialEq)] +enum RewriteState { + /// The expression is unchanged. + Unchanged, + /// Some part of the expression has been rewritten + RewrittenValid, + /// Some part of the expression has been rewritten, but some column + /// references could not be. + RewrittenInvalid, +} + +/// The function operates in two modes: +/// +/// 1) When `sync_with_child` is `true`: +/// +/// The function updates the indices of `expr` if the expression resides +/// in the input plan. For instance, given the expressions `a@1 + b@2` +/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are +/// updated to `a@0 + b@1` and `c@2`. +/// +/// 2) When `sync_with_child` is `false`: +/// +/// The function determines how the expression would be updated if a projection +/// was placed before the plan associated with the expression. If the expression +/// cannot be rewritten after the projection, it returns `None`. For example, +/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with +/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes +/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. +fn update_expr( + expr: &Arc, + projected_exprs: &[(Arc, String)], + sync_with_child: bool, +) -> Result>> { + let mut state = RewriteState::Unchanged; + let new_expr = expr + .clone() + .transform_up_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + if sync_with_child { + state = RewriteState::RewrittenValid; + // Update the index of `column`: + Ok(Transformed::Yes(projected_exprs[column.index()].0.clone())) + } else { + // default to invalid, in case we can't find the relevant column + state = RewriteState::RewrittenInvalid; + // Determine how to update `column` to accommodate `projected_exprs` + projected_exprs + .iter() + .enumerate() + .find_map(|(index, (projected_expr, alias))| { + projected_expr.as_any().downcast_ref::().and_then( + |projected_column| { + column.name().eq(projected_column.name()).then(|| { + state = RewriteState::RewrittenValid; + Arc::new(Column::new(alias, index)) as _ + }) + }, + ) + }) + .map_or_else( + || Ok(Transformed::No(expr)), + |c| Ok(Transformed::Yes(c)), + ) + } + }); + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) +} + +/// Given mapping representing the initial and new index values, +/// it updates the indices of columns in the [`PhysicalExpr`]. +fn update_column_index( + expr: &Arc, + mapping: &HashMap, +) -> Arc { + let mut state = RewriteState::Unchanged; + let new_expr = expr + .clone() + .transform_up_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + state = RewriteState::RewrittenValid; + // Update the index of `column`: + if let Some(updated) = mapping.get(column) { + Ok(Transformed::Yes(Arc::new(updated.clone()) as _)) + } else { + Ok(Transformed::No(expr.clone())) + } + }) + .unwrap(); + new_expr +} + +/// Collects all fields of the schema for a given plan in [`Column`] form. +fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { + plan.schema() + .fields() + .iter() + .enumerate() + .map(|(i, f)| Column::new(f.name(), i)) + .collect() +} + +/// Collects all columns in the join's equivalence and non-equivalence conditions as they are seen at the join output. +/// This means that columns from left table appear as they are, and right table column indices increased by left table size. +fn collect_columns_in_join_conditions( + on: &[(Arc, Arc)], + filter: Option<&JoinFilter>, + left_size: usize, + join_left_schema: SchemaRef, + join_right_schema: SchemaRef, +) -> HashSet { + let equivalence_columns = on + .iter() + .flat_map(|(col_left, col_right)| { + let left_columns = collect_columns(col_left); + let right_columns = collect_columns(col_right); + let mut state = RewriteState::Unchanged; + let right_columns = right_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect_vec(); + left_columns.into_iter().chain(right_columns).collect_vec() + }) + .collect::>(); + let non_equivalence_columns = filter + .map(|filter| { + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => Column::new( + join_left_schema.fields()[col_idx.index].name(), + col_idx.index, + ), + JoinSide::Right => Column::new( + join_right_schema.fields()[col_idx.index].name(), + col_idx.index + left_size, + ), + }) + .collect::>() + }) + .unwrap_or_default(); + equivalence_columns + .into_iter() + .chain(non_equivalence_columns.into_iter()) + .collect() +} + +/// Updates the equivalence conditions of the joins according to the new indices of columns. +fn update_equivalence_conditions( + on: &[(Arc, Arc)], + requirement_map_left: &ColumnRequirements, + requirement_map_right: &ColumnRequirements, +) -> JoinOn { + on.iter() + .map(|(left_col, right_col)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left_col + .clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new(Column::new( + column.name(), + column.index() + - removed_column_count( + requirement_map_left, + column.index(), + ), + )))) + }) + .unwrap(), + right_col + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new(Column::new( + column.name(), + column.index() + - removed_column_count( + requirement_map_right, + column.index(), + ), + )))) + }) + .unwrap(), + ) + }) + .collect() +} + +/// Updates the [`JoinFilter`] according to the new indices of columns. +fn update_non_equivalence_conditions( + filter: Option<&JoinFilter>, + requirement_map_left: &ColumnRequirements, + requirement_map_right: &ColumnRequirements, +) -> Option { + filter.map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: col_idx.index + - removed_column_count(requirement_map_left, col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: col_idx.index + - removed_column_count(requirement_map_right, col_idx.index), + side: JoinSide::Right, + }, + }) + .collect(), + filter.schema().clone(), + ) + }) +} + +/// Calculates how many index of the given column decreases becasue of +/// the removed columns which reside on the left side of that given column. +fn removed_column_count( + requirement_map: &ColumnRequirements, + column_index: usize, +) -> usize { + let mut left_skipped_columns = 0; + for unused_col in + requirement_map.iter().filter_map( + |(col, used)| { + if *used { + None + } else { + Some(col) + } + }, + ) + { + if unused_col.index() < column_index { + left_skipped_columns += 1; + } + } + left_skipped_columns +} + +fn rewrite_projection( + projection: &ProjectionExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + ProjectionExec::try_new( + projection + .expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.clone())) + .collect::>(), + input_plan, + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_filter( + predicate: &Arc, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + FilterExec::try_new(update_column_index(predicate, mapping), input_plan) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_repartition( + partitioning: &Partitioning, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { + let new_exprs = exprs + .iter() + .map(|expr| update_column_index(expr, &mapping)) + .collect::>(); + Partitioning::Hash(new_exprs, *size) + } else { + partitioning.clone() + }; + RepartitionExec::try_new(input_plan, new_partitioning).map(|plan| Arc::new(plan) as _) +} + +fn rewrite_sort( + sort: &SortExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + let new_sort_exprs = sort + .expr() + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &mapping), + options: sort_expr.options, + }) + .collect::>(); + Ok(Arc::new( + SortExec::new(new_sort_exprs, input_plan) + .with_fetch(sort.fetch()) + .with_preserve_partitioning(sort.preserve_partitioning()), + ) as _) +} + +fn rewrite_sort_preserving_merge( + sort: &SortPreservingMergeExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + let new_sort_exprs = sort + .expr() + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, &mapping), + options: sort_expr.options, + }) + .collect::>(); + Ok(Arc::new( + SortPreservingMergeExec::new(new_sort_exprs, input_plan).with_fetch(sort.fetch()), + ) as _) +} + +fn rewrite_hash_join( + hj: &HashJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_on = hj + .on() + .into_iter() + .map(|(left, right)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left.clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + right + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + ) + }) + .collect(); + let new_filter = hj.filter().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + HashJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + hj.join_type(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_nested_loop_join( + nlj: &NestedLoopJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_filter = nlj.filter().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + NestedLoopJoinExec::try_new( + left_input_plan, + right_input_plan, + new_filter, + nlj.join_type(), + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_sort_merge_join( + smj: &SortMergeJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_on = smj + .on() + .into_iter() + .map(|(left, right)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left.clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + right + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index( + &(Arc::new(column.clone()) as _), + &mapping, + ) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + ) + }) + .collect(); + let new_filter = smj.filter.as_ref().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + SortMergeJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + smj.join_type(), + smj.sort_options.clone(), + smj.null_equals_null, + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_symmetric_hash_join( + shj: &SymmetricHashJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + mapping: &HashMap, + left_size: usize, +) -> Result> { + let new_on = shj + .on() + .into_iter() + .map(|(left, right)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left.clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index(&(left.clone()), &mapping) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + right + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::No(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::No(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::Yes(Arc::new( + update_column_index(&(right.clone()), &mapping) + .as_any() + .downcast_ref::() + .unwrap() + .clone(), + ))) + }) + .unwrap(), + ) + }) + .collect(); + let new_filter = shj.filter().map(|filter| { + JoinFilter::new( + filter.expression().clone(), + filter + .column_indices() + .iter() + .map(|col_idx| match col_idx.side { + JoinSide::Left => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + JoinSide::Right => ColumnIndex { + index: mapping + .iter() + .find(|(old_column, _new_column)| { + old_column.index() == col_idx.index + left_size + }) + .map(|(_old_column, new_column)| new_column.index()) + .unwrap_or(col_idx.index), + side: JoinSide::Left, + }, + }) + .collect(), + filter.schema().clone(), + ) + }); + SymmetricHashJoinExec::try_new( + left_input_plan, + right_input_plan, + new_on, + new_filter, + shj.join_type(), + shj.null_equals_null(), + // TODO: update these + shj.left_sort_exprs().map(|exprs| exprs.to_vec()), + shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + shj.partition_mode(), + ) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_aggregate( + agg: &AggregateExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result>> { + let new_group_by = PhysicalGroupBy::new( + agg.group_expr() + .expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .collect(), + agg.group_expr() + .null_expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .collect(), + agg.group_expr().groups().to_vec(), + ); + let new_agg_expr = if let Some(new_agg_expr) = agg + .aggr_expr() + .iter() + .map(|aggr_expr| { + aggr_expr.clone().with_new_expressions( + aggr_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() + { + new_agg_expr + } else { + return Ok(None); + }; + let new_filter = agg + .filter_expr() + .iter() + .map(|opt_expr| { + opt_expr + .clone() + .map(|expr| update_column_index(&expr, mapping)) + }) + .collect(); + AggregateExec::try_new( + *agg.mode(), + new_group_by, + new_agg_expr, + new_filter, + input_plan, + agg.input_schema(), + ) + .map(|plan| Some(Arc::new(plan) as _)) +} + +fn rewrite_window_aggregate( + w_agg: &WindowAggExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result>> { + let new_window = if let Some(new_window) = w_agg + .window_expr() + .iter() + .map(|window_expr| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() + { + new_window + } else { + return Ok(None); + }; + let new_partition_keys = w_agg + .partition_keys + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(); + WindowAggExec::try_new(new_window, input_plan, new_partition_keys) + .map(|plan| Some(Arc::new(plan) as _)) +} + +fn rewrite_bounded_window_aggregate( + bw_agg: &BoundedWindowAggExec, + input_plan: Arc, + mapping: &HashMap, +) -> Result>> { + let new_window = if let Some(new_window) = bw_agg + .window_expr() + .iter() + .map(|window_expr| { + window_expr.clone().with_new_expressions( + window_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() + { + new_window + } else { + return Ok(None); + }; + let new_partition_keys = bw_agg + .partition_keys + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(); + BoundedWindowAggExec::try_new( + new_window, + input_plan, + new_partition_keys, + bw_agg.input_order_mode.clone(), + ) + .map(|plan| Some(Arc::new(plan) as _)) +} + +#[cfg(test)] +mod tests { + use std::sync::Arc; + + use crate::datasource::file_format::file_compression_type::FileCompressionType; + use crate::datasource::listing::PartitionedFile; + use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; + use crate::execution::context::SessionContext; + use crate::physical_optimizer::optimize_projections::{ + update_expr, OptimizeProjections, + }; + use crate::physical_optimizer::PhysicalOptimizerRule; + use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; + use crate::physical_plan::filter::FilterExec; + use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; + use crate::physical_plan::joins::StreamJoinPartitionMode; + use crate::physical_plan::projection::ProjectionExec; + use crate::physical_plan::repartition::RepartitionExec; + use crate::physical_plan::sorts::sort::SortExec; + use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + use crate::physical_plan::ExecutionPlan; + + use arrow::util::pretty::print_batches; + use arrow_schema::{DataType, Field, Schema, SortOptions}; + use datafusion_common::config::ConfigOptions; + use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; + use datafusion_execution::config::SessionConfig; + use datafusion_execution::object_store::ObjectStoreUrl; + use datafusion_expr::{ColumnarValue, Operator}; + use datafusion_physical_expr::expressions::{ + BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, + }; + use datafusion_physical_expr::{ + Partitioning, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, + }; + use datafusion_physical_plan::get_plan_string; + use datafusion_physical_plan::joins::SymmetricHashJoinExec; + use datafusion_physical_plan::union::UnionExec; + + use super::print_plan; + + fn create_simple_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::new_unknown(&schema), + projection: Some(vec![0, 1, 2, 3, 4]), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![vec![]], + }, + false, + 0, + 0, + None, + FileCompressionType::UNCOMPRESSED, + )) + } + + fn create_projecting_csv_exec() -> Arc { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Int32, true), + ])); + Arc::new(CsvExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::new_unknown(&schema), + projection: Some(vec![3, 0, 1]), + limit: None, + table_partition_cols: vec![], + output_ordering: vec![vec![]], + }, + false, + 0, + 0, + None, + FileCompressionType::UNCOMPRESSED, + )) + } + + #[test] + fn test_update_matching_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let child: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("b", 1)), "b".to_owned()), + (Arc::new(Column::new("d", 3)), "d".to_owned()), + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("f", 5)), "f".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + ]; + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &child, true)? + .unwrap() + .eq(&expected_expr)); + } + Ok(()) + } + + #[test] + fn test_update_projected_exprs() -> Result<()> { + let exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Divide, + Arc::new(Column::new("e", 5)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 3)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Divide, + Arc::new(Column::new("c", 0)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Divide, + Arc::new(Column::new("b", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d", 2))), + vec![ + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 2)), + Operator::Plus, + Arc::new(Column::new("e", 5)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 3)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 5)), + Operator::Plus, + Arc::new(Column::new("d", 2)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 3)), + Operator::Modulo, + Arc::new(Column::new("e", 5)), + ))), + )?), + ]; + let projected_exprs: Vec<(Arc, String)> = vec![ + (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("b", 1)), "b_new".to_owned()), + (Arc::new(Column::new("c", 2)), "c".to_owned()), + (Arc::new(Column::new("d", 3)), "d_new".to_owned()), + (Arc::new(Column::new("e", 4)), "e".to_owned()), + (Arc::new(Column::new("f", 5)), "f_new".to_owned()), + ]; + let expected_exprs: Vec> = vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Divide, + Arc::new(Column::new("e", 4)), + )), + Arc::new(CastExpr::new( + Arc::new(Column::new("a", 0)), + DataType::Float32, + None, + )), + Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), + Arc::new(ScalarFunctionExpr::new( + "scalar_expr", + Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + vec![ + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_new", 1)), + Operator::Divide, + Arc::new(Column::new("c", 2)), + )), + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Divide, + Arc::new(Column::new("b_new", 1)), + )), + ], + DataType::Int32, + None, + false, + )), + Arc::new(CaseExpr::try_new( + Some(Arc::new(Column::new("d_new", 3))), + vec![ + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d_new", 3)), + Operator::Plus, + Arc::new(Column::new("e", 4)), + )) as Arc, + ), + ( + Arc::new(Column::new("a", 0)) as Arc, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("e", 4)), + Operator::Plus, + Arc::new(Column::new("d_new", 3)), + )) as Arc, + ), + ], + Some(Arc::new(BinaryExpr::new( + Arc::new(Column::new("a", 0)), + Operator::Modulo, + Arc::new(Column::new("e", 4)), + ))), + )?), + ]; + for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { + assert!(update_expr(&expr, &projected_exprs, false)? + .unwrap() + .eq(&expected_expr)); + } + Ok(()) + } + + #[test] + fn test_csv_after_projection() -> Result<()> { + let csv = create_projecting_csv_exec(); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 2)), "b".to_string()), + (Arc::new(Column::new("d", 0)), "d".to_string()), + ], + csv.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@2 as b, d@0 as d]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[d, a, b], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_projection_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let child_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("e", 4)), "new_e".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("b", 1)), "new_b".to_string()), + ], + csv.clone(), + )?); + let top_projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), + ( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 0)), + Operator::Plus, + Arc::new(Column::new("new_e", 1)), + )), + "binary".to_string(), + ), + (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), + ], + child_projection.clone(), + )?); + let initial = get_plan_string(&top_projection); + let expected_initial = [ + "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", + " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(top_projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_coalesce_partitions_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let coalesce_partitions: Arc = + Arc::new(CoalescePartitionsExec::new(csv)); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + coalesce_partitions, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", + " CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@2 as d]", + " CoalescePartitionsExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false", + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_filter_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let predicate = Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b", 1)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + Operator::Gt, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("d", 3)), + Operator::Minus, + Arc::new(Column::new("a", 0)), + )), + )); + let filter: Arc = + Arc::new(FilterExec::try_new(predicate, csv)?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("a", 0)), "a_new".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + (Arc::new(Column::new("d", 3)), "d".to_string()), + ], + filter.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", + " FilterExec: b@1 - a@0 > d@3 - a@0", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@2 as d]", + " FilterExec: b@1 - a@0 > d@2 - a@0", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_join_after_projection() -> Result<()> { + let left_csv = create_simple_csv_exec(); + let right_csv = create_simple_csv_exec(); + let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( + left_csv, + right_csv, + vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], + // b_left-(1+a_right)<=a_right+c_left + Some(JoinFilter::new( + Arc::new(BinaryExpr::new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("b_left_inter", 0)), + Operator::Minus, + Arc::new(BinaryExpr::new( + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), + Operator::Plus, + Arc::new(Column::new("a_right_inter", 1)), + )), + )), + Operator::LtEq, + Arc::new(BinaryExpr::new( + Arc::new(Column::new("a_right_inter", 1)), + Operator::Plus, + Arc::new(Column::new("c_left_inter", 2)), + )), + )), + vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ], + Schema::new(vec![ + Field::new("b_left_inter", DataType::Int32, true), + Field::new("a_right_inter", DataType::Int32, true), + Field::new("c_left_inter", DataType::Int32, true), + ]), + )), + &JoinType::Inner, + true, + None, + None, + StreamJoinPartitionMode::SinglePartition, + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), + (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), + (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), + (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), + (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), + ], + join, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@3 as a_from_right, c@4 as c_from_right]", + " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + let expected_filter_col_ind = vec![ + ColumnIndex { + index: 1, + side: JoinSide::Left, + }, + ColumnIndex { + index: 0, + side: JoinSide::Right, + }, + ColumnIndex { + index: 2, + side: JoinSide::Left, + }, + ]; + assert_eq!( + expected_filter_col_ind, + after_optimize.children()[0] + .as_any() + .downcast_ref::() + .unwrap() + .filter() + .unwrap() + .column_indices() + ); + Ok(()) + } + + #[test] + fn test_repartition_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let repartition: Arc = Arc::new(RepartitionExec::try_new( + csv, + Partitioning::Hash( + vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("d", 3)), + ], + 6, + ), + )?); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("b", 1)), "b_new".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + (Arc::new(Column::new("d", 3)), "d_new".to_string()), + ], + repartition, + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", + " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@2 as d_new]", + " RepartitionExec: partitioning=Hash([a@0, b@1, d@2], 6), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_sort_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ], + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_sort_preserving_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("a", 0)), + )), + options: SortOptions::default(), + }, + ], + csv.clone(), + )); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + sort_req.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + + let expected = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_union_after_projection() -> Result<()> { + let csv = create_simple_csv_exec(); + let union: Arc = + Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); + let projection: Arc = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("a", 0)), "new_a".to_string()), + (Arc::new(Column::new("b", 1)), "b".to_string()), + ], + union.clone(), + )?); + let initial = get_plan_string(&projection); + let expected_initial = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" + ]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" + ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[test] + fn test_optimize_projections_filter_sort() -> Result<()> { + /* + INITIAL PLAN: + FilterExec(sum > 0): |sum@0 | + ProjectionExec: |c@2+x@0 as sum | + ProjectionExec: |x@2 |x@0 |c@1 | + SortExec(c@1, x@2): |x@0 |c@1 |x@2 | + ProjectionExec: |x@1 |c@0 |a@2 as x | + ProjectionExec: |c@2 |e@4 as x |a@0 | + CsvExec: |a |b |c |d |e | + ============================================================================================================= + OPTIMIZED PLAN: + FilterExec(sum > 0): |sum@0 | + ProjectionExec: |c@0+x@1 as sum | + SortExec(c@0, x@1): |c@0 |x@1 | + ProjectionExec: |c@2 |a@0 as x | + CsvExec: |a |b |c |d |e | + */ + let csv = create_simple_csv_exec(); + let projection1 = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("c", 2)), "c".to_string()), + (Arc::new(Column::new("e", 4)), "x".to_string()), + (Arc::new(Column::new("a", 0)), "a".to_string()), + ], + csv, + )?); + let projection2 = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("x", 1)), "x".to_string()), + (Arc::new(Column::new("c", 0)), "c".to_string()), + (Arc::new(Column::new("a", 2)), "x".to_string()), + ], + projection1, + )?); + let sort = Arc::new(SortExec::new( + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("c", 1)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("x", 2)), + options: SortOptions::default(), + }, + ], + projection2, + )); + let projection3 = Arc::new(ProjectionExec::try_new( + vec![ + (Arc::new(Column::new("x", 2)), "x".to_string()), + (Arc::new(Column::new("x", 0)), "x".to_string()), + (Arc::new(Column::new("c", 1)), "c".to_string()), + ], + sort, + )?); + let projection4 = Arc::new(ProjectionExec::try_new( + vec![( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("c", 2)), + Operator::Plus, + Arc::new(Column::new("x", 0)), + )), + "sum".to_string(), + )], + projection3, + )?); + let filter = Arc::new(FilterExec::try_new( + Arc::new(BinaryExpr::new( + Arc::new(Column::new("sum", 0)), + Operator::Gt, + Arc::new(Literal::new(ScalarValue::Int32(Some(0)))), + )), + projection4, + )?) as Arc; + let initial = get_plan_string(&filter); + let expected_initial = [ + "FilterExec: sum@0 > 0", + " ProjectionExec: expr=[c@2 + x@0 as sum]", + " ProjectionExec: expr=[x@2 as x, x@0 as x, c@1 as c]", + " SortExec: expr=[c@1 ASC,x@2 ASC]", + " ProjectionExec: expr=[x@1 as x, c@0 as c, a@2 as x]", + " ProjectionExec: expr=[c@2 as c, e@4 as x, a@0 as a]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(initial, expected_initial); + let after_optimize = + OptimizeProjections::new().optimize(filter, &ConfigOptions::new())?; + let expected = [ + "FilterExec: sum@0 > 0", + " ProjectionExec: expr=[c@0 + x@1 as sum]", + " SortExec: expr=[c@0 ASC,x@1 ASC]", + " ProjectionExec: expr=[c@2 as c, a@0 as x]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) + } + + #[tokio::test] + async fn test_trivial() -> Result<()> { + let mut config = SessionConfig::new() + .with_target_partitions(2) + .with_batch_size(4096); + let ctx = SessionContext::with_config(config); + let _dataframe = ctx + .sql( + "CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL + ) + STORED AS CSV + WITH HEADER ROW + LOCATION '/Users/berkaysahin/Desktop/datafusion-upstream/testing/data/csv/aggregate_test_100.csv'", + ) + .await?; + + let dataframe = ctx + .sql( + "WITH indices AS ( + SELECT 1 AS idx UNION ALL + SELECT 2 AS idx UNION ALL + SELECT 3 AS idx UNION ALL + SELECT 4 AS idx UNION ALL + SELECT 5 AS idx +) +SELECT data.arr[indices.idx] as element, array_length(data.arr) as array_len, dummy +FROM ( + SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100 +) data + CROSS JOIN indices +ORDER BY 1", + ) + .await?; + let physical_plan = dataframe.clone().create_physical_plan().await?; + let batches = dataframe.collect().await?; + let _ = print_plan(&physical_plan); + let _ = print_batches(&batches); + Ok(()) + } +} diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index f8c82576e254..ab7fe59e36e8 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,7 +19,6 @@ use std::sync::Arc; -use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; @@ -28,6 +27,7 @@ use crate::physical_optimizer::enforce_distribution::EnforceDistribution; use crate::physical_optimizer::enforce_sorting::EnforceSorting; use crate::physical_optimizer::join_selection::JoinSelection; use crate::physical_optimizer::limited_distinct_aggregation::LimitedDistinctAggregation; +use crate::physical_optimizer::optimize_projections::OptimizeProjections; use crate::physical_optimizer::output_requirements::OutputRequirements; use crate::physical_optimizer::pipeline_checker::PipelineChecker; use crate::physical_optimizer::topk_aggregation::TopKAggregation; @@ -113,13 +113,10 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), - // The ProjectionPushdown rule tries to push projections towards - // the sources in the execution plan. As a result of this process, - // a projection can disappear if it reaches the source providers, and - // sequential projections can merge into one. Even if these two cases - // are not present, the load of executors such as join or union will be - // reduced by narrowing their input tables. - Arc::new(ProjectionPushdown::new()), + // OptimizeProjections rule aims achieving the most effective use of projections + // in plans. It ensures that query plans are free from unnecessary projections + // and that no unused columns are propagated unnecessarily between plans. + Arc::new(OptimizeProjections::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs deleted file mode 100644 index 79d22374f9c2..000000000000 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ /dev/null @@ -1,2446 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! This file implements the `ProjectionPushdown` physical optimization rule. -//! The function [`remove_unnecessary_projections`] tries to push down all -//! projections one by one if the operator below is amenable to this. If a -//! projection reaches a source, it can even dissappear from the plan entirely. - -use std::collections::HashMap; -use std::sync::Arc; - -use super::output_requirements::OutputRequirementExec; -use super::PhysicalOptimizerRule; -use crate::datasource::physical_plan::CsvExec; -use crate::error::Result; -use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; -use crate::physical_plan::joins::{ - CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, - SymmetricHashJoinExec, -}; -use crate::physical_plan::memory::MemoryExec; -use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::{Distribution, ExecutionPlan}; - -use arrow_schema::SchemaRef; -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{DataFusionError, JoinSide}; -use datafusion_physical_expr::expressions::{Column, Literal}; -use datafusion_physical_expr::{ - Partitioning, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, - PhysicalSortRequirement, -}; -use datafusion_physical_plan::streaming::StreamingTableExec; -use datafusion_physical_plan::union::UnionExec; - -use itertools::Itertools; - -/// This rule inspects [`ProjectionExec`]'s in the given physical plan and tries to -/// remove or swap with its child. -#[derive(Default)] -pub struct ProjectionPushdown {} - -impl ProjectionPushdown { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for ProjectionPushdown { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - plan.transform_down(&remove_unnecessary_projections) - } - - fn name(&self) -> &str { - "ProjectionPushdown" - } - - fn schema_check(&self) -> bool { - true - } -} - -/// This function checks if `plan` is a [`ProjectionExec`], and inspects its -/// input(s) to test whether it can push `plan` under its input(s). This function -/// will operate on the entire tree and may ultimately remove `plan` entirely -/// by leveraging source providers with built-in projection capabilities. -pub fn remove_unnecessary_projections( - plan: Arc, -) -> Result>> { - let maybe_modified = if let Some(projection) = - plan.as_any().downcast_ref::() - { - // If the projection does not cause any change on the input, we can - // safely remove it: - if is_projection_removable(projection) { - return Ok(Transformed::Yes(projection.input().clone())); - } - // If it does, check if we can push it under its child(ren): - let input = projection.input().as_any(); - if let Some(csv) = input.downcast_ref::() { - try_swapping_with_csv(projection, csv) - } else if let Some(memory) = input.downcast_ref::() { - try_swapping_with_memory(projection, memory)? - } else if let Some(child_projection) = input.downcast_ref::() { - let maybe_unified = try_unifying_projections(projection, child_projection)?; - return if let Some(new_plan) = maybe_unified { - // To unify 3 or more sequential projections: - remove_unnecessary_projections(new_plan) - } else { - Ok(Transformed::No(plan)) - }; - } else if let Some(output_req) = input.downcast_ref::() { - try_swapping_with_output_req(projection, output_req)? - } else if input.is::() { - try_swapping_with_coalesce_partitions(projection)? - } else if let Some(filter) = input.downcast_ref::() { - try_swapping_with_filter(projection, filter)? - } else if let Some(repartition) = input.downcast_ref::() { - try_swapping_with_repartition(projection, repartition)? - } else if let Some(sort) = input.downcast_ref::() { - try_swapping_with_sort(projection, sort)? - } else if let Some(spm) = input.downcast_ref::() { - try_swapping_with_sort_preserving_merge(projection, spm)? - } else if let Some(union) = input.downcast_ref::() { - try_pushdown_through_union(projection, union)? - } else if let Some(hash_join) = input.downcast_ref::() { - try_pushdown_through_hash_join(projection, hash_join)? - } else if let Some(cross_join) = input.downcast_ref::() { - try_swapping_with_cross_join(projection, cross_join)? - } else if let Some(nl_join) = input.downcast_ref::() { - try_swapping_with_nested_loop_join(projection, nl_join)? - } else if let Some(sm_join) = input.downcast_ref::() { - try_swapping_with_sort_merge_join(projection, sm_join)? - } else if let Some(sym_join) = input.downcast_ref::() { - try_swapping_with_sym_hash_join(projection, sym_join)? - } else if let Some(ste) = input.downcast_ref::() { - try_swapping_with_streaming_table(projection, ste)? - } else { - // If the input plan of the projection is not one of the above, we - // conservatively assume that pushing the projection down may hurt. - // When adding new operators, consider adding them here if you - // think pushing projections under them is beneficial. - None - } - } else { - return Ok(Transformed::No(plan)); - }; - - Ok(maybe_modified.map_or(Transformed::No(plan), Transformed::Yes)) -} - -/// Tries to embed `projection` to its input (`csv`). If possible, returns -/// [`CsvExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_csv( - projection: &ProjectionExec, - csv: &CsvExec, -) -> Option> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into CsvExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()).then(|| { - let mut file_scan = csv.base_config().clone(); - let new_projections = new_projections_for_columns( - projection, - &file_scan - .projection - .unwrap_or((0..csv.schema().fields().len()).collect()), - ); - file_scan.projection = Some(new_projections); - - Arc::new(CsvExec::new( - file_scan, - csv.has_header(), - csv.delimiter(), - csv.quote(), - csv.escape(), - csv.file_compression_type, - )) as _ - }) -} - -/// Tries to embed `projection` to its input (`memory`). If possible, returns -/// [`MemoryExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_memory( - projection: &ProjectionExec, - memory: &MemoryExec, -) -> Result>> { - // If there is any non-column or alias-carrier expression, Projection should not be removed. - // This process can be moved into MemoryExec, but it would be an overlap of their responsibility. - all_alias_free_columns(projection.expr()) - .then(|| { - let all_projections = (0..memory.schema().fields().len()).collect(); - let new_projections = new_projections_for_columns( - projection, - memory.projection().as_ref().unwrap_or(&all_projections), - ); - - MemoryExec::try_new( - memory.partitions(), - memory.original_schema(), - Some(new_projections), - ) - .map(|e| Arc::new(e) as _) - }) - .transpose() -} - -/// Tries to embed `projection` to its input (`streaming table`). -/// If possible, returns [`StreamingTableExec`] as the top plan. Otherwise, -/// returns `None`. -fn try_swapping_with_streaming_table( - projection: &ProjectionExec, - streaming_table: &StreamingTableExec, -) -> Result>> { - if !all_alias_free_columns(projection.expr()) { - return Ok(None); - } - - let streaming_table_projections = streaming_table - .projection() - .as_ref() - .map(|i| i.as_ref().to_vec()); - let new_projections = new_projections_for_columns( - projection, - &streaming_table_projections - .unwrap_or((0..streaming_table.schema().fields().len()).collect()), - ); - - let mut lex_orderings = vec![]; - for lex_ordering in streaming_table.projected_output_ordering().into_iter() { - let mut orderings = vec![]; - for order in lex_ordering { - let Some(new_ordering) = update_expr(&order.expr, projection.expr(), false)? - else { - return Ok(None); - }; - orderings.push(PhysicalSortExpr { - expr: new_ordering, - options: order.options, - }); - } - lex_orderings.push(orderings); - } - - StreamingTableExec::try_new( - streaming_table.partition_schema().clone(), - streaming_table.partitions().clone(), - Some(new_projections.as_ref()), - lex_orderings, - streaming_table.is_infinite(), - ) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Unifies `projection` with its input (which is also a [`ProjectionExec`]). -fn try_unifying_projections( - projection: &ProjectionExec, - child: &ProjectionExec, -) -> Result>> { - let mut projected_exprs = vec![]; - let mut column_ref_map: HashMap = HashMap::new(); - - // Collect the column references usage in the outer projection. - projection.expr().iter().for_each(|(expr, _)| { - expr.apply(&mut |expr| { - Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { - *column_ref_map.entry(column.clone()).or_default() += 1; - } - VisitRecursion::Continue - }) - }) - .unwrap(); - }); - - // Merging these projections is not beneficial, e.g - // If an expression is not trivial and it is referred more than 1, unifies projections will be - // beneficial as caching mechanism for non-trivial computations. - // See discussion in: https://github.com/apache/arrow-datafusion/issues/8296 - if column_ref_map.iter().any(|(column, count)| { - *count > 1 && !is_expr_trivial(&child.expr()[column.index()].0.clone()) - }) { - return Ok(None); - } - - for (expr, alias) in projection.expr() { - // If there is no match in the input projection, we cannot unify these - // projections. This case will arise if the projection expression contains - // a `PhysicalExpr` variant `update_expr` doesn't support. - let Some(expr) = update_expr(expr, child.expr(), true)? else { - return Ok(None); - }; - projected_exprs.push((expr, alias.clone())); - } - - ProjectionExec::try_new(projected_exprs, child.input().clone()) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Checks if the given expression is trivial. -/// An expression is considered trivial if it is either a `Column` or a `Literal`. -fn is_expr_trivial(expr: &Arc) -> bool { - expr.as_any().downcast_ref::().is_some() - || expr.as_any().downcast_ref::().is_some() -} - -/// Tries to swap `projection` with its input (`output_req`). If possible, -/// performs the swap and returns [`OutputRequirementExec`] as the top plan. -/// Otherwise, returns `None`. -fn try_swapping_with_output_req( - projection: &ProjectionExec, - output_req: &OutputRequirementExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_sort_reqs = vec![]; - // None or empty_vec can be treated in the same way. - if let Some(reqs) = &output_req.required_input_ordering()[0] { - for req in reqs { - let Some(new_expr) = update_expr(&req.expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_sort_reqs.push(PhysicalSortRequirement { - expr: new_expr, - options: req.options, - }); - } - } - - let dist_req = match &output_req.required_input_distribution()[0] { - Distribution::HashPartitioned(exprs) => { - let mut updated_exprs = vec![]; - for expr in exprs { - let Some(new_expr) = update_expr(expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_exprs.push(new_expr); - } - Distribution::HashPartitioned(updated_exprs) - } - dist => dist.clone(), - }; - - make_with_child(projection, &output_req.input()) - .map(|input| { - OutputRequirementExec::new( - input, - (!updated_sort_reqs.is_empty()).then_some(updated_sort_reqs), - dist_req, - ) - }) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Tries to swap `projection` with its input, which is known to be a -/// [`CoalescePartitionsExec`]. If possible, performs the swap and returns -/// [`CoalescePartitionsExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_coalesce_partitions( - projection: &ProjectionExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - // CoalescePartitionsExec always has a single child, so zero indexing is safe. - make_with_child(projection, &projection.input().children()[0]) - .map(|e| Some(Arc::new(CoalescePartitionsExec::new(e)) as _)) -} - -/// Tries to swap `projection` with its input (`filter`). If possible, performs -/// the swap and returns [`FilterExec`] as the top plan. Otherwise, returns `None`. -fn try_swapping_with_filter( - projection: &ProjectionExec, - filter: &FilterExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down: - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - // Each column in the predicate expression must exist after the projection. - let Some(new_predicate) = update_expr(filter.predicate(), projection.expr(), false)? - else { - return Ok(None); - }; - - FilterExec::try_new(new_predicate, make_with_child(projection, filter.input())?) - .and_then(|e| { - let selectivity = filter.default_selectivity(); - e.with_default_selectivity(selectivity) - }) - .map(|e| Some(Arc::new(e) as _)) -} - -/// Tries to swap the projection with its input [`RepartitionExec`]. If it can be done, -/// it returns the new swapped version having the [`RepartitionExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_repartition( - projection: &ProjectionExec, - repartition: &RepartitionExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - // If pushdown is not beneficial or applicable, break it. - if projection.benefits_from_input_partitioning()[0] || !all_columns(projection.expr()) - { - return Ok(None); - } - - let new_projection = make_with_child(projection, repartition.input())?; - - let new_partitioning = match repartition.partitioning() { - Partitioning::Hash(partitions, size) => { - let mut new_partitions = vec![]; - for partition in partitions { - let Some(new_partition) = - update_expr(partition, projection.expr(), false)? - else { - return Ok(None); - }; - new_partitions.push(new_partition); - } - Partitioning::Hash(new_partitions, *size) - } - others => others.clone(), - }; - - Ok(Some(Arc::new(RepartitionExec::try_new( - new_projection, - new_partitioning, - )?))) -} - -/// Tries to swap the projection with its input [`SortExec`]. If it can be done, -/// it returns the new swapped version having the [`SortExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sort( - projection: &ProjectionExec, - sort: &SortExec, -) -> Result>> { - // If the projection does not narrow the the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_exprs = vec![]; - for sort in sort.expr() { - let Some(new_expr) = update_expr(&sort.expr, projection.expr(), false)? else { - return Ok(None); - }; - updated_exprs.push(PhysicalSortExpr { - expr: new_expr, - options: sort.options, - }); - } - - Ok(Some(Arc::new( - SortExec::new(updated_exprs, make_with_child(projection, sort.input())?) - .with_fetch(sort.fetch()) - .with_preserve_partitioning(sort.preserve_partitioning()), - ))) -} - -/// Tries to swap the projection with its input [`SortPreservingMergeExec`]. -/// If this is possible, it returns the new [`SortPreservingMergeExec`] whose -/// child is a projection. Otherwise, it returns None. -fn try_swapping_with_sort_preserving_merge( - projection: &ProjectionExec, - spm: &SortPreservingMergeExec, -) -> Result>> { - // If the projection does not narrow the schema, we should not try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let mut updated_exprs = vec![]; - for sort in spm.expr() { - let Some(updated_expr) = update_expr(&sort.expr, projection.expr(), false)? - else { - return Ok(None); - }; - updated_exprs.push(PhysicalSortExpr { - expr: updated_expr, - options: sort.options, - }); - } - - Ok(Some(Arc::new( - SortPreservingMergeExec::new( - updated_exprs, - make_with_child(projection, spm.input())?, - ) - .with_fetch(spm.fetch()), - ))) -} - -/// Tries to push `projection` down through `union`. If possible, performs the -/// pushdown and returns a new [`UnionExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_union( - projection: &ProjectionExec, - union: &UnionExec, -) -> Result>> { - // If the projection doesn't narrow the schema, we shouldn't try to push it down. - if projection.expr().len() >= projection.input().schema().fields().len() { - return Ok(None); - } - - let new_children = union - .children() - .into_iter() - .map(|child| make_with_child(projection, &child)) - .collect::>>()?; - - Ok(Some(Arc::new(UnionExec::new(new_children)))) -} - -/// Tries to push `projection` down through `hash_join`. If possible, performs the -/// pushdown and returns a new [`HashJoinExec`] as the top plan which has projections -/// as its children. Otherwise, returns `None`. -fn try_pushdown_through_hash_join( - projection: &ProjectionExec, - hash_join: &HashJoinExec, -) -> Result>> { - // Convert projected expressions to columns. We can not proceed if this is - // not possible. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - hash_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - hash_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - hash_join.on(), - ) else { - return Ok(None); - }; - - let new_filter = if let Some(filter) = hash_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - hash_join.left(), - hash_join.right(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - hash_join.left(), - hash_join.right(), - )?; - - Ok(Some(Arc::new(HashJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - new_filter, - hash_join.join_type(), - *hash_join.partition_mode(), - hash_join.null_equals_null, - )?))) -} - -/// Tries to swap the projection with its input [`CrossJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`CrossJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_cross_join( - projection: &ProjectionExec, - cross_join: &CrossJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - cross_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - cross_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - cross_join.left(), - cross_join.right(), - )?; - - Ok(Some(Arc::new(CrossJoinExec::new( - Arc::new(new_left), - Arc::new(new_right), - )))) -} - -/// Tries to swap the projection with its input [`NestedLoopJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`NestedLoopJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_nested_loop_join( - projection: &ProjectionExec, - nl_join: &NestedLoopJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - nl_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - nl_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let new_filter = if let Some(filter) = nl_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - nl_join.left(), - nl_join.right(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - nl_join.left(), - nl_join.right(), - )?; - - Ok(Some(Arc::new(NestedLoopJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_filter, - nl_join.join_type(), - )?))) -} - -/// Tries to swap the projection with its input [`SortMergeJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`SortMergeJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sort_merge_join( - projection: &ProjectionExec, - sm_join: &SortMergeJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - sm_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - sm_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - sm_join.on(), - ) else { - return Ok(None); - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - &sm_join.children()[0], - &sm_join.children()[1], - )?; - - Ok(Some(Arc::new(SortMergeJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - sm_join.filter.clone(), - sm_join.join_type, - sm_join.sort_options.clone(), - sm_join.null_equals_null, - )?))) -} - -/// Tries to swap the projection with its input [`SymmetricHashJoinExec`]. If it can be done, -/// it returns the new swapped version having the [`SymmetricHashJoinExec`] as the top plan. -/// Otherwise, it returns None. -fn try_swapping_with_sym_hash_join( - projection: &ProjectionExec, - sym_join: &SymmetricHashJoinExec, -) -> Result>> { - // Convert projected PhysicalExpr's to columns. If not possible, we cannot proceed. - let Some(projection_as_columns) = physical_to_column_exprs(projection.expr()) else { - return Ok(None); - }; - - let (far_right_left_col_ind, far_left_right_col_ind) = join_table_borders( - sym_join.left().schema().fields().len(), - &projection_as_columns, - ); - - if !join_allows_pushdown( - &projection_as_columns, - sym_join.schema(), - far_right_left_col_ind, - far_left_right_col_ind, - ) { - return Ok(None); - } - - let Some(new_on) = update_join_on( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - sym_join.on(), - ) else { - return Ok(None); - }; - - let new_filter = if let Some(filter) = sym_join.filter() { - match update_join_filter( - &projection_as_columns[0..=far_right_left_col_ind as _], - &projection_as_columns[far_left_right_col_ind as _..], - filter, - sym_join.left(), - sym_join.right(), - ) { - Some(updated_filter) => Some(updated_filter), - None => return Ok(None), - } - } else { - None - }; - - let (new_left, new_right) = new_join_children( - projection_as_columns, - far_right_left_col_ind, - far_left_right_col_ind, - sym_join.left(), - sym_join.right(), - )?; - - Ok(Some(Arc::new(SymmetricHashJoinExec::try_new( - Arc::new(new_left), - Arc::new(new_right), - new_on, - new_filter, - sym_join.join_type(), - sym_join.null_equals_null(), - sym_join.right().output_ordering().map(|p| p.to_vec()), - sym_join.left().output_ordering().map(|p| p.to_vec()), - sym_join.partition_mode(), - )?))) -} - -/// Compare the inputs and outputs of the projection. All expressions must be -/// columns without alias, and projection does not change the order of fields. -fn is_projection_removable(projection: &ProjectionExec) -> bool { - let exprs = projection.expr(); - exprs.iter().enumerate().all(|(idx, (expr, alias))| { - let Some(col) = expr.as_any().downcast_ref::() else { - return false; - }; - col.name() == alias && col.index() == idx - }) && exprs.len() == projection.input().schema().fields().len() -} - -/// Given the expression set of a projection, checks if the projection causes -/// any renaming or constructs a non-`Column` physical expression. -fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|column| column.name() == alias) - .unwrap_or(false) - }) -} - -/// Updates a source provider's projected columns according to the given -/// projection operator's expressions. To use this function safely, one must -/// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns( - projection: &ProjectionExec, - source: &[usize], -) -> Vec { - projection - .expr() - .iter() - .filter_map(|(expr, _)| { - expr.as_any() - .downcast_ref::() - .map(|expr| source[expr.index()]) - }) - .collect() -} - -/// The function operates in two modes: -/// -/// 1) When `sync_with_child` is `true`: -/// -/// The function updates the indices of `expr` if the expression resides -/// in the input plan. For instance, given the expressions `a@1 + b@2` -/// and `c@0` with the input schema `c@2, a@0, b@1`, the expressions are -/// updated to `a@0 + b@1` and `c@2`. -/// -/// 2) When `sync_with_child` is `false`: -/// -/// The function determines how the expression would be updated if a projection -/// was placed before the plan associated with the expression. If the expression -/// cannot be rewritten after the projection, it returns `None`. For example, -/// given the expressions `c@0`, `a@1` and `b@2`, and the [`ProjectionExec`] with -/// an output schema of `a, c_new`, then `c@0` becomes `c_new@1`, `a@1` becomes -/// `a@0`, but `b@2` results in `None` since the projection does not include `b`. -fn update_expr( - expr: &Arc, - projected_exprs: &[(Arc, String)], - sync_with_child: bool, -) -> Result>> { - #[derive(Debug, PartialEq)] - enum RewriteState { - /// The expression is unchanged. - Unchanged, - /// Some part of the expression has been rewritten - RewrittenValid, - /// Some part of the expression has been rewritten, but some column - /// references could not be. - RewrittenInvalid, - } - - let mut state = RewriteState::Unchanged; - - let new_expr = expr - .clone() - .transform_up_mut(&mut |expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - if sync_with_child { - state = RewriteState::RewrittenValid; - // Update the index of `column`: - Ok(Transformed::Yes(projected_exprs[column.index()].0.clone())) - } else { - // default to invalid, in case we can't find the relevant column - state = RewriteState::RewrittenInvalid; - // Determine how to update `column` to accommodate `projected_exprs` - projected_exprs - .iter() - .enumerate() - .find_map(|(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - (column.name().eq(projected_column.name()) - && column.index() == projected_column.index()) - .then(|| { - state = RewriteState::RewrittenValid; - Arc::new(Column::new(alias, index)) as _ - }) - }, - ) - }) - .map_or_else( - || Ok(Transformed::No(expr)), - |c| Ok(Transformed::Yes(c)), - ) - } - }); - - new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) -} - -/// Creates a new [`ProjectionExec`] instance with the given child plan and -/// projected expressions. -fn make_with_child( - projection: &ProjectionExec, - child: &Arc, -) -> Result> { - ProjectionExec::try_new(projection.expr().to_vec(), child.clone()) - .map(|e| Arc::new(e) as _) -} - -/// Returns `true` if all the expressions in the argument are `Column`s. -fn all_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, _)| expr.as_any().is::()) -} - -/// Downcasts all the expressions in `exprs` to `Column`s. If any of the given -/// expressions is not a `Column`, returns `None`. -fn physical_to_column_exprs( - exprs: &[(Arc, String)], -) -> Option> { - exprs - .iter() - .map(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|col| (col.clone(), alias.clone())) - }) - .collect() -} - -/// Returns the last index before encountering a column coming from the right table when traveling -/// through the projection from left to right, and the last index before encountering a column -/// coming from the left table when traveling through the projection from right to left. -/// If there is no column in the projection coming from the left side, it returns (-1, ...), -/// if there is no column in the projection coming from the right side, it returns (..., projection length). -fn join_table_borders( - left_table_column_count: usize, - projection_as_columns: &[(Column, String)], -) -> (i32, i32) { - let far_right_left_col_ind = projection_as_columns - .iter() - .enumerate() - .take_while(|(_, (projection_column, _))| { - projection_column.index() < left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(-1); - - let far_left_right_col_ind = projection_as_columns - .iter() - .enumerate() - .rev() - .take_while(|(_, (projection_column, _))| { - projection_column.index() >= left_table_column_count - }) - .last() - .map(|(index, _)| index as i32) - .unwrap_or(projection_as_columns.len() as i32); - - (far_right_left_col_ind, far_left_right_col_ind) -} - -/// Tries to update the equi-join `Column`'s of a join as if the the input of -/// the join was replaced by a projection. -fn update_join_on( - proj_left_exprs: &[(Column, String)], - proj_right_exprs: &[(Column, String)], - hash_join_on: &[(PhysicalExprRef, PhysicalExprRef)], -) -> Option> { - // TODO: Clippy wants the "map" call removed, but doing so generates - // a compilation error. Remove the clippy directive once this - // issue is fixed. - #[allow(clippy::map_identity)] - let (left_idx, right_idx): (Vec<_>, Vec<_>) = hash_join_on - .iter() - .map(|(left, right)| (left, right)) - .unzip(); - - let new_left_columns = new_columns_for_join_on(&left_idx, proj_left_exprs); - let new_right_columns = new_columns_for_join_on(&right_idx, proj_right_exprs); - - match (new_left_columns, new_right_columns) { - (Some(left), Some(right)) => Some(left.into_iter().zip(right).collect()), - _ => None, - } -} - -/// This function generates a new set of columns to be used in a hash join -/// operation based on a set of equi-join conditions (`hash_join_on`) and a -/// list of projection expressions (`projection_exprs`). -fn new_columns_for_join_on( - hash_join_on: &[&PhysicalExprRef], - projection_exprs: &[(Column, String)], -) -> Option> { - let new_columns = hash_join_on - .iter() - .filter_map(|on| { - // Rewrite all columns in `on` - (*on) - .clone() - .transform(&|expr| { - if let Some(column) = expr.as_any().downcast_ref::() { - // Find the column in the projection expressions - let new_column = projection_exprs - .iter() - .enumerate() - .find(|(_, (proj_column, _))| { - column.name() == proj_column.name() - }) - .map(|(index, (_, alias))| Column::new(alias, index)); - if let Some(new_column) = new_column { - Ok(Transformed::Yes(Arc::new(new_column))) - } else { - // If the column is not found in the projection expressions, - // it means that the column is not projected. In this case, - // we cannot push the projection down. - Err(DataFusionError::Internal(format!( - "Column {:?} not found in projection expressions", - column - ))) - } - } else { - Ok(Transformed::No(expr)) - } - }) - .ok() - }) - .collect::>(); - (new_columns.len() == hash_join_on.len()).then_some(new_columns) -} - -/// Tries to update the column indices of a [`JoinFilter`] as if the the input of -/// the join was replaced by a projection. -fn update_join_filter( - projection_left_exprs: &[(Column, String)], - projection_right_exprs: &[(Column, String)], - join_filter: &JoinFilter, - join_left: &Arc, - join_right: &Arc, -) -> Option { - let mut new_left_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Left, - projection_left_exprs, - join_left.schema(), - ) - .into_iter(); - let mut new_right_indices = new_indices_for_join_filter( - join_filter, - JoinSide::Right, - projection_right_exprs, - join_right.schema(), - ) - .into_iter(); - - // Check if all columns match: - (new_right_indices.len() + new_left_indices.len() - == join_filter.column_indices().len()) - .then(|| { - JoinFilter::new( - join_filter.expression().clone(), - join_filter - .column_indices() - .iter() - .map(|col_idx| ColumnIndex { - index: if col_idx.side == JoinSide::Left { - new_left_indices.next().unwrap() - } else { - new_right_indices.next().unwrap() - }, - side: col_idx.side, - }) - .collect(), - join_filter.schema().clone(), - ) - }) -} - -/// This function determines and returns a vector of indices representing the -/// positions of columns in `projection_exprs` that are involved in `join_filter`, -/// and correspond to a particular side (`join_side`) of the join operation. -fn new_indices_for_join_filter( - join_filter: &JoinFilter, - join_side: JoinSide, - projection_exprs: &[(Column, String)], - join_child_schema: SchemaRef, -) -> Vec { - join_filter - .column_indices() - .iter() - .filter(|col_idx| col_idx.side == join_side) - .filter_map(|col_idx| { - projection_exprs.iter().position(|(col, _)| { - col.name() == join_child_schema.fields()[col_idx.index].name() - }) - }) - .collect() -} - -/// Checks three conditions for pushing a projection down through a join: -/// - Projection must narrow the join output schema. -/// - Columns coming from left/right tables must be collected at the left/right -/// sides of the output table. -/// - Left or right table is not lost after the projection. -fn join_allows_pushdown( - projection_as_columns: &[(Column, String)], - join_schema: SchemaRef, - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, -) -> bool { - // Projection must narrow the join output: - projection_as_columns.len() < join_schema.fields().len() - // Are the columns from different tables mixed? - && (far_right_left_col_ind + 1 == far_left_right_col_ind) - // Left or right table is not lost after the projection. - && far_right_left_col_ind >= 0 - && far_left_right_col_ind < projection_as_columns.len() as i32 -} - -/// If pushing down the projection over this join's children seems possible, -/// this function constructs the new [`ProjectionExec`]s that will come on top -/// of the original children of the join. -fn new_join_children( - projection_as_columns: Vec<(Column, String)>, - far_right_left_col_ind: i32, - far_left_right_col_ind: i32, - left_child: &Arc, - right_child: &Arc, -) -> Result<(ProjectionExec, ProjectionExec)> { - let new_left = ProjectionExec::try_new( - projection_as_columns[0..=far_right_left_col_ind as _] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new(col.name(), col.index())) as _, - alias.clone(), - ) - }) - .collect_vec(), - left_child.clone(), - )?; - let left_size = left_child.schema().fields().len() as i32; - let new_right = ProjectionExec::try_new( - projection_as_columns[far_left_right_col_ind as _..] - .iter() - .map(|(col, alias)| { - ( - Arc::new(Column::new( - col.name(), - // Align projected expressions coming from the right - // table with the new right child projection: - (col.index() as i32 - left_size) as _, - )) as _, - alias.clone(), - ) - }) - .collect_vec(), - right_child.clone(), - )?; - - Ok((new_left, new_right)) -} - -#[cfg(test)] -mod tests { - use std::sync::Arc; - - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; - use crate::physical_optimizer::output_requirements::OutputRequirementExec; - use crate::physical_optimizer::projection_pushdown::{ - join_table_borders, update_expr, ProjectionPushdown, - }; - use crate::physical_optimizer::PhysicalOptimizerRule; - use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::joins::utils::{ColumnIndex, JoinFilter}; - use crate::physical_plan::joins::StreamJoinPartitionMode; - use crate::physical_plan::memory::MemoryExec; - use crate::physical_plan::projection::ProjectionExec; - use crate::physical_plan::repartition::RepartitionExec; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use crate::physical_plan::{get_plan_string, ExecutionPlan}; - - use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; - use datafusion_common::config::ConfigOptions; - use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; - use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_execution::{SendableRecordBatchStream, TaskContext}; - use datafusion_expr::{ColumnarValue, Operator}; - use datafusion_physical_expr::expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, - }; - use datafusion_physical_expr::{ - Distribution, Partitioning, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, ScalarFunctionExpr, - }; - use datafusion_physical_plan::joins::SymmetricHashJoinExec; - use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; - use datafusion_physical_plan::union::UnionExec; - - use itertools::Itertools; - - #[test] - fn test_update_matching_exprs() -> Result<()> { - let exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Divide, - Arc::new(Column::new("e", 5)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 3)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 0)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 2))), - vec![ - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 2)), - Operator::Plus, - Arc::new(Column::new("e", 5)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 5)), - Operator::Plus, - Arc::new(Column::new("d", 2)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Modulo, - Arc::new(Column::new("e", 5)), - ))), - )?), - ]; - let child: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("c", 2)), "c".to_owned()), - (Arc::new(Column::new("b", 1)), "b".to_owned()), - (Arc::new(Column::new("d", 3)), "d".to_owned()), - (Arc::new(Column::new("a", 0)), "a".to_owned()), - (Arc::new(Column::new("f", 5)), "f".to_owned()), - (Arc::new(Column::new("e", 4)), "e".to_owned()), - ]; - - let expected_exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Divide, - Arc::new(Column::new("e", 4)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 0)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 2)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 3))), - vec![ - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 3)), - Operator::Plus, - Arc::new(Column::new("e", 4)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 4)), - Operator::Plus, - Arc::new(Column::new("d", 3)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Modulo, - Arc::new(Column::new("e", 4)), - ))), - )?), - ]; - - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &child, true)? - .unwrap() - .eq(&expected_expr)); - } - - Ok(()) - } - - #[test] - fn test_update_projected_exprs() -> Result<()> { - let exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Divide, - Arc::new(Column::new("e", 5)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 3)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Divide, - Arc::new(Column::new("c", 0)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Divide, - Arc::new(Column::new("b", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d", 2))), - vec![ - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 2)), - Operator::Plus, - Arc::new(Column::new("e", 5)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 3)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 5)), - Operator::Plus, - Arc::new(Column::new("d", 2)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 3)), - Operator::Modulo, - Arc::new(Column::new("e", 5)), - ))), - )?), - ]; - let projected_exprs: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("a", 3)), "a".to_owned()), - (Arc::new(Column::new("b", 1)), "b_new".to_owned()), - (Arc::new(Column::new("c", 0)), "c".to_owned()), - (Arc::new(Column::new("d", 2)), "d_new".to_owned()), - (Arc::new(Column::new("e", 5)), "e".to_owned()), - (Arc::new(Column::new("f", 4)), "f_new".to_owned()), - ]; - - let expected_exprs: Vec> = vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Divide, - Arc::new(Column::new("e", 4)), - )), - Arc::new(CastExpr::new( - Arc::new(Column::new("a", 0)), - DataType::Float32, - None, - )), - Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), - Arc::new(ScalarFunctionExpr::new( - "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), - vec![ - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_new", 1)), - Operator::Divide, - Arc::new(Column::new("c", 2)), - )), - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Divide, - Arc::new(Column::new("b_new", 1)), - )), - ], - DataType::Int32, - None, - false, - )), - Arc::new(CaseExpr::try_new( - Some(Arc::new(Column::new("d_new", 3))), - vec![ - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d_new", 3)), - Operator::Plus, - Arc::new(Column::new("e", 4)), - )) as Arc, - ), - ( - Arc::new(Column::new("a", 0)) as Arc, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("e", 4)), - Operator::Plus, - Arc::new(Column::new("d_new", 3)), - )) as Arc, - ), - ], - Some(Arc::new(BinaryExpr::new( - Arc::new(Column::new("a", 0)), - Operator::Modulo, - Arc::new(Column::new("e", 4)), - ))), - )?), - ]; - - for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &projected_exprs, false)? - .unwrap() - .eq(&expected_expr)); - } - - Ok(()) - } - - #[test] - fn test_join_table_borders() -> Result<()> { - let projections = vec![ - (Column::new("b", 1), "b".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("c", 2), "c".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("h", 7), "h".to_owned()), - (Column::new("g", 6), "g".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (4, 5) - ); - - let left_table_column_count = 8; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (7, 8) - ); - - let left_table_column_count = 1; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (-1, 0) - ); - - let projections = vec![ - (Column::new("a", 0), "a".to_owned()), - (Column::new("b", 1), "b".to_owned()), - (Column::new("d", 3), "d".to_owned()), - (Column::new("g", 6), "g".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("f", 5), "f".to_owned()), - (Column::new("e", 4), "e".to_owned()), - (Column::new("h", 7), "h".to_owned()), - ]; - let left_table_column_count = 5; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (2, 7) - ); - - let left_table_column_count = 7; - assert_eq!( - join_table_borders(left_table_column_count, &projections), - (6, 7) - ); - - Ok(()) - } - - fn create_simple_csv_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])); - Arc::new(CsvExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema.clone(), - file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_unknown(&schema), - projection: Some(vec![0, 1, 2, 3, 4]), - limit: None, - table_partition_cols: vec![], - output_ordering: vec![vec![]], - }, - false, - 0, - 0, - None, - FileCompressionType::UNCOMPRESSED, - )) - } - - fn create_projecting_csv_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - ])); - Arc::new(CsvExec::new( - FileScanConfig { - object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), - file_schema: schema.clone(), - file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], - statistics: Statistics::new_unknown(&schema), - projection: Some(vec![3, 2, 1]), - limit: None, - table_partition_cols: vec![], - output_ordering: vec![vec![]], - }, - false, - 0, - 0, - None, - FileCompressionType::UNCOMPRESSED, - )) - } - - fn create_projecting_memory_exec() -> Arc { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])); - - Arc::new(MemoryExec::try_new(&[], schema, Some(vec![2, 0, 3, 4])).unwrap()) - } - - #[test] - fn test_csv_after_projection() -> Result<()> { - let csv = create_projecting_csv_exec(); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 2)), "b".to_string()), - (Arc::new(Column::new("d", 0)), "d".to_string()), - ], - csv.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@2 as b, d@0 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[d, c, b], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "CsvExec: file_groups={1 group: [[x]]}, projection=[b, d], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_memory_after_projection() -> Result<()> { - let memory = create_projecting_memory_exec(); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("d", 2)), "d".to_string()), - (Arc::new(Column::new("e", 3)), "e".to_string()), - (Arc::new(Column::new("a", 1)), "a".to_string()), - ], - memory.clone(), - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[d@2 as d, e@3 as e, a@1 as a]", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = ["MemoryExec: partitions=0, partition_sizes=[]"]; - assert_eq!(get_plan_string(&after_optimize), expected); - assert_eq!( - after_optimize - .clone() - .as_any() - .downcast_ref::() - .unwrap() - .projection() - .clone() - .unwrap(), - vec![3, 4, 0] - ); - - Ok(()) - } - - #[test] - fn test_streaming_table_after_projection() -> Result<()> { - struct DummyStreamPartition { - schema: SchemaRef, - } - impl PartitionStream for DummyStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } - } - - let streaming_table = StreamingTableExec::try_new( - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])), - vec![Arc::new(DummyStreamPartition { - schema: Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])), - }) as _], - Some(&vec![0_usize, 2, 4, 3]), - vec![ - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("e", 2)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - ], - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), - options: SortOptions::default(), - }], - ] - .into_iter(), - true, - )?; - let projection = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("d", 3)), "d".to_string()), - (Arc::new(Column::new("e", 2)), "e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - ], - Arc::new(streaming_table) as _, - )?) as _; - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let result = after_optimize - .as_any() - .downcast_ref::() - .unwrap(); - assert_eq!( - result.partition_schema(), - &Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - ])) - ); - assert_eq!( - result.projection().clone().unwrap().to_vec(), - vec![3_usize, 4, 0] - ); - assert_eq!( - result.projected_schema(), - &Schema::new(vec![ - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Int32, true), - Field::new("a", DataType::Int32, true), - ]) - ); - assert_eq!( - result.projected_output_ordering().into_iter().collect_vec(), - vec![ - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("e", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 2)), - options: SortOptions::default(), - }, - ], - vec![PhysicalSortExpr { - expr: Arc::new(Column::new("d", 0)), - options: SortOptions::default(), - }], - ] - ); - assert!(result.is_infinite()); - - Ok(()) - } - - #[test] - fn test_projection_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let child_projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("e", 4)), "new_e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "new_b".to_string()), - ], - csv.clone(), - )?); - let top_projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("new_b", 3)), "new_b".to_string()), - ( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_e", 1)), - )), - "binary".to_string(), - ), - (Arc::new(Column::new("new_b", 3)), "newest_b".to_string()), - ], - child_projection.clone(), - )?); - - let initial = get_plan_string(&top_projection); - let expected_initial = [ - "ProjectionExec: expr=[new_b@3 as new_b, c@0 + new_e@1 as binary, new_b@3 as newest_b]", - " ProjectionExec: expr=[c@2 as c, e@4 as new_e, a@0 as a, b@1 as new_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(top_projection, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_output_req_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(OutputRequirementExec::new( - csv.clone(), - Some(vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 1)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: Some(SortOptions::default()), - }, - ]), - Distribution::HashPartitioned(vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " OutputRequirementExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected: [&str; 3] = [ - "OutputRequirementExec", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - - assert_eq!(get_plan_string(&after_optimize), expected); - let expected_reqs = vec![ - PhysicalSortRequirement { - expr: Arc::new(Column::new("b", 2)), - options: Some(SortOptions::default()), - }, - PhysicalSortRequirement { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 0)), - Operator::Plus, - Arc::new(Column::new("new_a", 1)), - )), - options: Some(SortOptions::default()), - }, - ]; - assert_eq!( - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_ordering()[0] - .clone() - .unwrap(), - expected_reqs - ); - let expected_distribution: Vec> = vec![ - Arc::new(Column::new("new_a", 1)), - Arc::new(Column::new("b", 2)), - ]; - if let Distribution::HashPartitioned(vec) = after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .required_input_distribution()[0] - .clone() - { - assert!(vec - .iter() - .zip(expected_distribution) - .all(|(actual, expected)| actual.eq(&expected))); - } else { - panic!("Expected HashPartitioned distribution!"); - }; - - Ok(()) - } - - #[test] - fn test_coalesce_partitions_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let coalesce_partitions: Arc = - Arc::new(CoalescePartitionsExec::new(csv)); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - ], - coalesce_partitions, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CoalescePartitionsExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "CoalescePartitionsExec", - " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_filter_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let predicate = Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b", 1)), - Operator::Minus, - Arc::new(Column::new("a", 0)), - )), - Operator::Gt, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("d", 3)), - Operator::Minus, - Arc::new(Column::new("a", 0)), - )), - )); - let filter: Arc = - Arc::new(FilterExec::try_new(predicate, csv)?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - ], - filter.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " FilterExec: b@1 - a@0 > d@3 - a@0", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "FilterExec: b@1 - a_new@0 > d@2 - a_new@0", - " ProjectionExec: expr=[a@0 as a_new, b@1 as b, d@3 as d]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_join_after_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ]), - )), - &JoinType::Inner, - true, - None, - None, - StreamJoinPartitionMode::SinglePartition, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c_from_left".to_string()), - (Arc::new(Column::new("b", 1)), "b_from_left".to_string()), - (Arc::new(Column::new("a", 0)), "a_from_left".to_string()), - (Arc::new(Column::new("a", 5)), "a_from_right".to_string()), - (Arc::new(Column::new("c", 7)), "c_from_right".to_string()), - ], - join, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left, a@5 as a_from_right, c@7 as c_from_right]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b_from_left@1, c_from_right@1)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " ProjectionExec: expr=[c@2 as c_from_left, b@1 as b_from_left, a@0 as a_from_left]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[a@0 as a_from_right, c@2 as c_from_right]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - let expected_filter_col_ind = vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 0, - side: JoinSide::Left, - }, - ]; - - assert_eq!( - expected_filter_col_ind, - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .filter() - .unwrap() - .column_indices() - ); - - Ok(()) - } - - #[test] - fn test_join_after_required_projection() -> Result<()> { - let left_csv = create_simple_csv_exec(); - let right_csv = create_simple_csv_exec(); - - let join: Arc = Arc::new(SymmetricHashJoinExec::try_new( - left_csv, - right_csv, - vec![(Arc::new(Column::new("b", 1)), Arc::new(Column::new("c", 2)))], - // b_left-(1+a_right)<=a_right+c_left - Some(JoinFilter::new( - Arc::new(BinaryExpr::new( - Arc::new(BinaryExpr::new( - Arc::new(Column::new("b_left_inter", 0)), - Operator::Minus, - Arc::new(BinaryExpr::new( - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))), - Operator::Plus, - Arc::new(Column::new("a_right_inter", 1)), - )), - )), - Operator::LtEq, - Arc::new(BinaryExpr::new( - Arc::new(Column::new("a_right_inter", 1)), - Operator::Plus, - Arc::new(Column::new("c_left_inter", 2)), - )), - )), - vec![ - ColumnIndex { - index: 1, - side: JoinSide::Left, - }, - ColumnIndex { - index: 0, - side: JoinSide::Right, - }, - ColumnIndex { - index: 2, - side: JoinSide::Left, - }, - ], - Schema::new(vec![ - Field::new("b_left_inter", DataType::Int32, true), - Field::new("a_right_inter", DataType::Int32, true), - Field::new("c_left_inter", DataType::Int32, true), - ]), - )), - &JoinType::Inner, - true, - None, - None, - StreamJoinPartitionMode::SinglePartition, - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("a", 5)), "a".to_string()), - (Arc::new(Column::new("b", 6)), "b".to_string()), - (Arc::new(Column::new("c", 7)), "c".to_string()), - (Arc::new(Column::new("d", 8)), "d".to_string()), - (Arc::new(Column::new("e", 9)), "e".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("d", 3)), "d".to_string()), - (Arc::new(Column::new("e", 4)), "e".to_string()), - ], - join, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "ProjectionExec: expr=[a@5 as a, b@6 as b, c@7 as c, d@8 as d, e@9 as e, a@0 as a, b@1 as b, c@2 as c, d@3 as d, e@4 as e]", - " SymmetricHashJoinExec: mode=SinglePartition, join_type=Inner, on=[(b@1, c@2)], filter=b_left_inter@0 - 1 + a_right_inter@1 <= a_right_inter@1 + c_left_inter@2", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - Ok(()) - } - - #[test] - fn test_repartition_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let repartition: Arc = Arc::new(RepartitionExec::try_new( - csv, - Partitioning::Hash( - vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("d", 3)), - ], - 6, - ), - )?); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - (Arc::new(Column::new("a", 0)), "a".to_string()), - (Arc::new(Column::new("d", 3)), "d_new".to_string()), - ], - repartition, - )?); - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " RepartitionExec: partitioning=Hash([a@0, b@1, d@3], 6), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "RepartitionExec: partitioning=Hash([a@1, b_new@0, d_new@2], 6), input_partitions=1", - " ProjectionExec: expr=[b@1 as b_new, a@0 as a, d@3 as d_new]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - assert_eq!( - after_optimize - .as_any() - .downcast_ref::() - .unwrap() - .partitioning() - .clone(), - Partitioning::Hash( - vec![ - Arc::new(Column::new("a", 1)), - Arc::new(Column::new("b_new", 0)), - Arc::new(Column::new("d_new", 2)), - ], - 6, - ), - ); - - Ok(()) - } - - #[test] - fn test_sort_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortExec::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, - ], - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortExec: expr=[b@1 ASC,c@2 + a@0 ASC]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SortExec: expr=[b@2 ASC,c@0 + new_a@1 ASC]", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_sort_preserving_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let sort_req: Arc = Arc::new(SortPreservingMergeExec::new( - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(BinaryExpr::new( - Arc::new(Column::new("c", 2)), - Operator::Plus, - Arc::new(Column::new("a", 0)), - )), - options: SortOptions::default(), - }, - ], - csv.clone(), - )); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - sort_req.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " SortPreservingMergeExec: [b@1 ASC,c@2 + a@0 ASC]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "SortPreservingMergeExec: [b@2 ASC,c@0 + new_a@1 ASC]", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } - - #[test] - fn test_union_after_projection() -> Result<()> { - let csv = create_simple_csv_exec(); - let union: Arc = - Arc::new(UnionExec::new(vec![csv.clone(), csv.clone(), csv])); - let projection: Arc = Arc::new(ProjectionExec::try_new( - vec![ - (Arc::new(Column::new("c", 2)), "c".to_string()), - (Arc::new(Column::new("a", 0)), "new_a".to_string()), - (Arc::new(Column::new("b", 1)), "b".to_string()), - ], - union.clone(), - )?); - - let initial = get_plan_string(&projection); - let expected_initial = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " UnionExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(initial, expected_initial); - - let after_optimize = - ProjectionPushdown::new().optimize(projection, &ConfigOptions::new())?; - - let expected = [ - "UnionExec", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - " ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; - assert_eq!(get_plan_string(&after_optimize), expected); - - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index dd0261420304..278a8895164d 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -70,7 +70,7 @@ impl TopKAggregation { // We found what we want: clone, copy the limit down, and return modified node let new_aggr = AggregateExec::try_new( *aggr.mode(), - aggr.group_by().clone(), + aggr.group_expr().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), aggr.input().clone(), diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index a1d9a02cf6b1..fa160f2a117e 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -656,18 +656,18 @@ async fn test_physical_plan_display_indent_multi_children() { let dataframe = ctx.sql(sql).await.unwrap(); let physical_plan = dataframe.create_physical_plan().await.unwrap(); let expected = vec![ - "ProjectionExec: expr=[c1@0 as c1]", - " CoalesceBatchesExec: target_batch_size=4096", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + "CoalesceBatchesExec: target_batch_size=4096", + " ProjectionExec: expr=[c1@0 as c1]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", + " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " ProjectionExec: expr=[c1@0 as c2]", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", + " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " ProjectionExec: expr=[c1@0 as c2]", + " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true" ]; let normalizer = ExplainNormalizer::new(); diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index 2bb205ce90dc..c4a1aa7fbb5f 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -133,6 +133,15 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { fn create_sliding_accumulator(&self) -> Result> { not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") } + + /// Rewrites the aggregate expression with the given expressions. + /// The order of the given expressions is taken into account while replacing. + fn with_new_expressions( + self: Arc, + expressions: Vec>, + ) -> Option> { + None + } } /// Checks whether the given aggregate expression is order-sensitive. diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index 6cf2810ce588..9b42cca76bd0 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -153,6 +153,19 @@ impl AggregateExpr for Sum { } downcast_sum!(self, helper) } + + fn with_new_expressions( + self: Arc, + mut expressions: Vec>, + ) -> Option> { + Some(Arc::new(Self { + name: self.name.clone(), + data_type: self.data_type.clone(), + return_type: self.return_type.clone(), + expr: expressions.swap_remove(0), + nullable: self.nullable, + })) + } } impl PartialEq for Sum { diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index e2714dc42bea..78b653a47b7e 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -27,7 +27,9 @@ use arrow::compute::kernels::sort::SortColumn; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; -use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; +use datafusion_common::{ + internal_err, not_impl_err, DataFusionError, Result, ScalarValue, +}; use datafusion_expr::window_state::{ PartitionBatchState, WindowAggState, WindowFrameContext, }; @@ -127,6 +129,15 @@ pub trait WindowExpr: Send + Sync + Debug { /// Get the reverse expression of this [WindowExpr]. fn get_reverse_expr(&self) -> Option>; + + /// Rewrites the window expression with the given expressions. + /// The order of the given expressions is taken into account while replacing. + fn with_new_expressions( + self: Arc, + expressions: Vec>, + ) -> Option> { + None + } } /// Extension trait that adds common functionality to [`AggregateWindowExpr`]s diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 156362430558..c116d4b96157 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -471,17 +471,13 @@ impl AggregateExec { } } - pub fn group_by(&self) -> &PhysicalGroupBy { - &self.group_by - } - /// true, if this Aggregate has a group-by with no required or explicit ordering, /// no filtering and no aggregate expressions /// This method qualifies the use of the LimitedDistinctAggregation rewrite rule /// on an AggregateExec. pub fn is_unordered_unfiltered_group_by_distinct(&self) -> bool { // ensure there is a group by - if self.group_by().is_empty() { + if self.group_expr().is_empty() { return false; } // ensure there are no aggregate expressions diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 1c4a6ac0ecaf..675b003cac4b 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -39,6 +39,7 @@ use datafusion_physical_expr::{ }; use futures::stream::TryStreamExt; +use projection::ProjectionExec; use tokio::task::JoinSet; mod topk; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index cc2ab62049ed..6c3d21955607 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -36,6 +36,7 @@ use crate::{ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; @@ -66,11 +67,33 @@ pub struct ProjectionExec { impl ProjectionExec { /// Create a projection on an input pub fn try_new( - expr: Vec<(Arc, String)>, + mut expr: Vec<(Arc, String)>, input: Arc, ) -> Result { let input_schema = input.schema(); - + expr = expr + .iter() + .enumerate() + .map(|(expr_idx, (expression, name))| { + expression + .clone() + .transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + }) + .map(|source_expr| (source_expr, name.to_string())) + }) + .collect::>>()?; let fields: Result> = expr .iter() .map(|(e, name)| { @@ -94,7 +117,6 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(&expr, &input_schema)?; - let input_eqs = input.equivalence_properties(); let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); let output_ordering = project_eqs.oeq_class().output_ordering(); diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 6b9db5589391..172db0bc0570 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -137,7 +137,8 @@ Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.accou physical_plan SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST] --RecursiveQueryExec: name=balances, is_distinct=false -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true +----ProjectionExec: expr=[time@0 as time, name@1 as name, account_balance@2 as account_balance] +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true ----CoalescePartitionsExec ------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] --------CoalesceBatchesExec: target_batch_size=2 diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 3f609e254839..d897d7c733d4 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -94,7 +94,7 @@ Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 A ----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] ------TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] +ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3)@1 as c3, FIRST_VALUE(aggregate_test_100.c2)@2 as c2] --SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ----SortExec: expr=[c1@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 4002164cc918..d9f9fde45e93 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,8 +43,8 @@ Projection: aggregate_test_100.c1 --Filter: aggregate_test_100.c2 > Int8(10) ----TableScan: aggregate_test_100 projection=[c1, c2], partial_filters=[aggregate_test_100.c2 > Int8(10)] physical_plan -ProjectionExec: expr=[c1@0 as c1] ---CoalesceBatchesExec: target_batch_size=8192 +CoalesceBatchesExec: target_batch_size=8192 +--ProjectionExec: expr=[c1@0 as c1] ----FilterExec: c2@1 > 10 ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true @@ -252,7 +252,7 @@ physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -309,7 +309,7 @@ GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Co --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -344,7 +344,7 @@ GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after OptimizeProjections SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 1312f2916ed6..84c884b86716 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,8 +57,8 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 -----ProjectionExec: expr=[a@1 as a] -------CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=8192 +------ProjectionExec: expr=[a@1 as a] --------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true ----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -102,12 +102,13 @@ GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 ----ProjectionExec: expr=[a@0 as a2, b@1 as b] ------CoalesceBatchesExec: target_batch_size=8192 ---------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true -----------CoalesceBatchesExec: target_batch_size=8192 -------------FilterExec: d@3 = 3 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +--------ProjectionExec: expr=[a@0 as a, b@1 as b] +----------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +------------CoalesceBatchesExec: target_batch_size=8192 +--------------FilterExec: d@3 = 3 +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index e063d6e8960a..3748253be046 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -370,9 +370,9 @@ AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------ProjectionExec: expr=[] -----------GlobalLimitExec: skip=6, fetch=3 -------------CoalesceBatchesExec: target_batch_size=8192 +--------GlobalLimitExec: skip=6, fetch=3 +----------CoalesceBatchesExec: target_batch_size=8192 +------------ProjectionExec: expr=[] --------------FilterExec: a@0 > 3 ----------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index ba407f6d2852..8a05e49192fd 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -635,8 +635,8 @@ Projection: lineitem.l_partkey ----Filter: (part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) ------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] physical_plan -ProjectionExec: expr=[l_partkey@0 as l_partkey] ---CoalesceBatchesExec: target_batch_size=8192 +CoalesceBatchesExec: target_batch_size=8192 +--ProjectionExec: expr=[l_partkey@0 as l_partkey] ----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15 ------CoalesceBatchesExec: target_batch_size=8192 --------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 @@ -729,11 +729,11 @@ Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG ------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] ---ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] -----CoalesceBatchesExec: target_batch_size=8192 +--CoalesceBatchesExec: target_batch_size=8192 +----ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)] ---------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] -----------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 @@ -741,8 +741,8 @@ AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM --------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ----------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 ------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index b7bbc0706576..0b0b77f0af5e 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1364,6 +1364,7 @@ AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[COUNT(*)] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true + statement ok drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index fa4445d4cd4c..fbe2b847a54a 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1752,8 +1752,8 @@ ProjectionExec: expr=[COUNT(*)@0 as global_count] --------------CoalesceBatchesExec: target_batch_size=4096 ----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] ---------------------ProjectionExec: expr=[c1@0 as c1] -----------------------CoalesceBatchesExec: target_batch_size=4096 +--------------------CoalesceBatchesExec: target_batch_size=4096 +----------------------ProjectionExec: expr=[c1@0 as c1] ------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 --------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true @@ -2814,13 +2814,13 @@ Projection: sum1, sum2, count1, count2 ------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col --------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] +----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] +--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] + query IIII SELECT @@ -2861,14 +2861,12 @@ Projection: sum1, sum2, count1, count2 ------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col --------------TableScan: annotated_data_infinite projection=[ts, inc_col] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, count1@2 as count1, count2@3 as count2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2, ts@1 as ts] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] -----------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -------------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] - +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] +----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] +--------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] query IIII SELECT From e8bb63336cc94ba3f84aacc07b1e348b6fb5141d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 13:05:19 +0300 Subject: [PATCH 02/85] Add new reproducer test --- .../optimize_projections.rs | 30 ++++++++++++++++++- datafusion/physical-plan/src/lib.rs | 1 - 2 files changed, 29 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 5363032aa954..5db3be0d1715 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -5009,7 +5009,7 @@ mod tests { ) STORED AS CSV WITH HEADER ROW - LOCATION '/Users/berkaysahin/Desktop/datafusion-upstream/testing/data/csv/aggregate_test_100.csv'", + LOCATION '../../testing/data/csv/aggregate_test_100.csv'", ) .await?; @@ -5036,4 +5036,32 @@ ORDER BY 1", let _ = print_batches(&batches); Ok(()) } + + /// TODO: Result of this query is wrong, where output schema orders is different what is should be. + /// The problem originates in `AggregateExec: mode=FinalPartitioned` at the top. + #[tokio::test] + async fn test_trivial2() -> Result<()> { + let mut config = SessionConfig::new() + .with_target_partitions(2) + .with_batch_size(4096); + let ctx = SessionContext::with_config(config); + let _dataframe = ctx + .sql( + "CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER) as VALUES (83,0,38), (26,0,79), (43,81,24)", + ) + .await?; + + let dataframe = ctx + .sql( + "SELECT DISTINCT * FROM tab0 AS cor0 GROUP BY cor0.col1, cor0.col2, cor0.col0", + ) + .await?; + + let physical_plan = dataframe.clone().create_physical_plan().await?; + let batches = dataframe.collect().await?; + let _ = print_plan(&physical_plan); + let _ = print_batches(&batches); + assert!(false); + Ok(()) + } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 675b003cac4b..1c4a6ac0ecaf 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -39,7 +39,6 @@ use datafusion_physical_expr::{ }; use futures::stream::TryStreamExt; -use projection::ProjectionExec; use tokio::task::JoinSet; mod topk; From fd29eeac395ec4c2338ff3d1d3f5481402d56122 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 13:30:55 +0300 Subject: [PATCH 03/85] Simplifications --- .../optimize_projections.rs | 156 ++++++------------ 1 file changed, 52 insertions(+), 104 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 5db3be0d1715..e95e79745c3b 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -2284,21 +2284,7 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; - let mut new_mapping = HashMap::new(); - for col in self.required_columns.iter() { - let mut skipped_columns = 0; - for unused_col in unused_columns.iter() { - if unused_col.index() < col.index() { - skipped_columns += 1; - } - } - if skipped_columns > 0 { - new_mapping.insert( - col.clone(), - Column::new(col.name(), col.index() - skipped_columns), - ); - } - } + let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); let new_requirements = collect_columns_in_plan_schema(&inserted_projection); let inserted_projection = ProjectionOptimizer { @@ -2346,21 +2332,7 @@ impl ProjectionOptimizer { }) .collect::>>()?; - let mut new_mapping = HashMap::new(); - for col in self.required_columns.iter() { - let mut skipped_columns = 0; - for unused_col in unused_columns.iter() { - if unused_col.index() < col.index() { - skipped_columns += 1; - } - } - if skipped_columns > 0 { - new_mapping.insert( - col.clone(), - Column::new(col.name(), col.index() - skipped_columns), - ); - } - } + let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); let new_requirements = inserted_projections .iter() @@ -2413,24 +2385,10 @@ impl ProjectionOptimizer { let required_columns = projected_exprs .iter() - .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) - .collect::>(); + .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap().clone()) + .collect::>(); - let mut new_mapping = HashMap::new(); - for col in required_columns.into_iter() { - let mut skipped_columns = 0; - for unused_col in unused_columns.iter() { - if unused_col.index() < col.index() { - skipped_columns += 1; - } - } - if skipped_columns > 0 { - new_mapping.insert( - col.clone(), - Column::new(col.name(), col.index() - skipped_columns), - ); - } - } + let new_mapping = calculate_column_mapping(&required_columns, &unused_columns); let required_columns = collect_columns_in_plan_schema(&inserted_projection); let inserted_projection = ProjectionOptimizer { @@ -2504,20 +2462,9 @@ impl ProjectionOptimizer { let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); - let mut unused_columns = HashSet::new(); + let (used_columns, mut unused_columns) = split_column_reqs(&base); + let projected_exprs = convert_projection_exprs(used_columns); - let projected_exprs = base - .into_iter() - .filter_map(|(col, used)| { - if used { - let col_name = col.name().to_string(); - Some((Arc::new(col) as Arc, col_name)) - } else { - unused_columns.insert(col); - None - } - }) - .collect(); window.iter().for_each(|(col, used)| { if !used { unused_columns.insert(col.clone()); @@ -2528,21 +2475,7 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; - let mut new_mapping = HashMap::new(); - for col in self.required_columns.iter() { - let mut skipped_columns = 0; - for unused_col in unused_columns.iter().chain(unused_columns.iter()) { - if unused_col.index() < col.index() { - skipped_columns += 1; - } - } - if skipped_columns > 0 { - new_mapping.insert( - col.clone(), - Column::new(col.name(), col.index() - skipped_columns), - ); - } - } + let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); let new_requirements = collect_columns_in_plan_schema(&inserted_projection); let inserted_projection = ProjectionOptimizer { @@ -2565,20 +2498,9 @@ impl ProjectionOptimizer { let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); - let mut unused_columns = HashSet::new(); - - let projected_exprs = base - .into_iter() - .filter_map(|(col, used)| { - if used { - let col_name = col.name().to_string(); - Some((Arc::new(col) as Arc, col_name)) - } else { - unused_columns.insert(col); - None - } - }) - .collect(); + // let mut unused_columns = HashSet::new(); + let (required_cols, mut unused_columns) = split_column_reqs(&base); + let projected_exprs = convert_projection_exprs(required_cols); window.iter().for_each(|(col, used)| { if !used { unused_columns.insert(col.clone()); @@ -2589,21 +2511,7 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; - let mut new_mapping = HashMap::new(); - for col in self.required_columns.iter() { - let mut skipped_columns = 0; - for unused_col in unused_columns.iter().chain(unused_columns.iter()) { - if unused_col.index() < col.index() { - skipped_columns += 1; - } - } - if skipped_columns > 0 { - new_mapping.insert( - col.clone(), - Column::new(col.name(), col.index() - skipped_columns), - ); - } - } + let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); let new_requirements = collect_columns_in_plan_schema(&inserted_projection); let inserted_projection = ProjectionOptimizer { @@ -3095,6 +3003,46 @@ impl TreeNode for ProjectionOptimizer { } } +fn calculate_column_mapping(required_columns: &HashSet, unused_columns: &HashSet) -> HashMap{ + let mut new_mapping = HashMap::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 +} + +fn split_column_reqs(reqs: &ColumnRequirements) -> (HashSet, HashSet) { + let mut required = HashSet::new(); + let mut unused = HashSet::new(); + for (col, is_req) in reqs { + if *is_req { + required.insert(col.clone()); + } else { + unused.insert(col.clone()); + } + } + (required, unused) +} + +fn convert_projection_exprs(cols: HashSet) -> Vec<(Arc, String)> { + let mut result = cols.into_iter().map(|col| { + let name = col.name().to_string(); + (Arc::new(col) as Arc, name) + }).collect::>(); + result +} + #[derive(Default)] pub struct OptimizeProjections {} From 4233028d4c352137ff3fef2ba303e76176d04167 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 13:39:21 +0300 Subject: [PATCH 04/85] Simplifications --- .../optimize_projections.rs | 85 +++++++------------ 1 file changed, 32 insertions(+), 53 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index e95e79745c3b..236d7fa94efb 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -2263,19 +2263,8 @@ impl ProjectionOptimizer { ) -> Result<(Self, HashMap)> { // During the iteration, we construct the ProjectionExec with required columns as the new child, // and also collect the unused columns to store the index changes after removal of some columns. - let mut unused_columns = HashSet::new(); - let mut projected_exprs = requirement_map - .into_iter() - .filter_map(|(col, used)| { - if used { - let col_name = col.name().to_string(); - Some((Arc::new(col) as Arc, col_name)) - } else { - unused_columns.insert(col); - None - } - }) - .collect::>(); + let (used_columns, unused_columns) = split_column_reqs(&requirement_map); + let mut projected_exprs = convert_projection_exprs(used_columns); projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() }); @@ -2284,7 +2273,8 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; - let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); + 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 { @@ -2304,19 +2294,8 @@ impl ProjectionOptimizer { ) -> Result<(Vec, HashMap)> { // During the iteration, we construct the ProjectionExec's with required columns as the new children, // and also collect the unused columns to store the index changes after removal of some columns. - let mut unused_columns = HashSet::new(); - let mut projected_exprs = requirement_map - .into_iter() - .filter_map(|(col, used)| { - if used { - let col_name = col.name().to_string(); - Some((Arc::new(col) as Arc, col_name)) - } else { - unused_columns.insert(col); - None - } - }) - .collect::>(); + let (used_columns, unused_columns) = split_column_reqs(&requirement_map); + let mut projected_exprs = convert_projection_exprs(used_columns); projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() }); @@ -2332,7 +2311,8 @@ impl ProjectionOptimizer { }) .collect::>>()?; - let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); + let new_mapping = + calculate_column_mapping(&self.required_columns, &unused_columns); let new_requirements = inserted_projections .iter() @@ -2360,21 +2340,10 @@ impl ProjectionOptimizer { requirement_map_left: ColumnRequirements, children_index: usize, ) -> Result<(Self, HashMap)> { - let mut unused_columns = HashSet::new(); // During the iteration, we construct the ProjectionExec with required columns as the new child, // and also collect the unused columns to store the index changes after removal of some columns. - let mut projected_exprs = requirement_map_left - .into_iter() - .filter_map(|(col, used)| { - if used { - let col_name = col.name().to_string(); - Some((Arc::new(col) as Arc, col_name)) - } else { - unused_columns.insert(col); - None - } - }) - .collect::>(); + let (used_columns, unused_columns) = split_column_reqs(&requirement_map_left); + let mut projected_exprs = convert_projection_exprs(used_columns); projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() }); @@ -2475,7 +2444,8 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; - let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); + 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 { @@ -2511,7 +2481,8 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; - let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); + 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 { @@ -3003,7 +2974,10 @@ impl TreeNode for ProjectionOptimizer { } } -fn calculate_column_mapping(required_columns: &HashSet, unused_columns: &HashSet) -> HashMap{ +fn calculate_column_mapping( + required_columns: &HashSet, + unused_columns: &HashSet, +) -> HashMap { let mut new_mapping = HashMap::new(); for col in required_columns.iter() { let mut skipped_columns = 0; @@ -3035,11 +3009,16 @@ fn split_column_reqs(reqs: &ColumnRequirements) -> (HashSet, HashSet) -> Vec<(Arc, String)> { - let mut result = cols.into_iter().map(|col| { - let name = col.name().to_string(); - (Arc::new(col) as Arc, name) - }).collect::>(); +fn convert_projection_exprs( + cols: HashSet, +) -> Vec<(Arc, String)> { + let mut result = cols + .into_iter() + .map(|col| { + let name = col.name().to_string(); + (Arc::new(col) as Arc, name) + }) + .collect::>(); result } @@ -4939,8 +4918,8 @@ mod tests { .with_batch_size(4096); let ctx = SessionContext::with_config(config); let _dataframe = ctx - .sql( - "CREATE EXTERNAL TABLE aggregate_test_100 ( + .sql( + "CREATE EXTERNAL TABLE aggregate_test_100 ( c1 VARCHAR NOT NULL, c2 TINYINT NOT NULL, c3 SMALLINT NOT NULL, @@ -4958,8 +4937,8 @@ mod tests { STORED AS CSV WITH HEADER ROW LOCATION '../../testing/data/csv/aggregate_test_100.csv'", - ) - .await?; + ) + .await?; let dataframe = ctx .sql( From 410f44a7b8016a581799499f5c6401309c21f27b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 16:47:51 +0300 Subject: [PATCH 05/85] Simplifications --- .../optimize_projections.rs | 52 ++++++++----------- 1 file changed, 23 insertions(+), 29 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 236d7fa94efb..98257b303c7f 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -69,7 +69,7 @@ use datafusion_common::DataFusionError; use datafusion_common::{internal_err, JoinSide, JoinType}; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{Partitioning, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::{LexOrdering, Partitioning, PhysicalExpr, PhysicalSortExpr}; use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -671,13 +671,8 @@ impl ProjectionOptimizer { // Rewrite the hashed expressions if there is any with possibly updated column indices. let new_partitioning = if let Partitioning::Hash(exprs, size) = repartition.partitioning() { - Partitioning::Hash( - exprs - .iter() - .map(|expr| update_column_index(expr, &schema_mapping)) - .collect::>(), - *size, - ) + let updated_exprs = update_exprs(exprs, &schema_mapping); + Partitioning::Hash(updated_exprs, *size) } else { repartition.partitioning().clone() }; @@ -711,14 +706,7 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. - let new_sort_exprs = sort - .expr() - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, &schema_mapping), - options: sort_expr.options, - }) - .collect::>(); + let new_sort_exprs = update_sort_exprs(sort.expr(), &schema_mapping); let plan = Arc::new( SortExec::new(new_sort_exprs, new_child.plan.clone()) .with_preserve_partitioning(sort.preserve_partitioning()) @@ -754,14 +742,7 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. - let new_sort_exprs = sortp_merge - .expr() - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, &schema_mapping), - options: sort_expr.options, - }) - .collect::>(); + let new_sort_exprs = update_sort_exprs(sortp_merge.expr(), &schema_mapping); let plan = Arc::new( SortPreservingMergeExec::new(new_sort_exprs, new_child.plan.clone()) .with_fetch(sortp_merge.fetch()), @@ -2050,12 +2031,9 @@ impl ProjectionOptimizer { .zip(window_usage.clone()) .filter(|(_window_expr, (_window_col, usage))| *usage) .map(|(window_expr, (_window_col, _usage))| { + let new_exprs = update_exprs(&window_expr.expressions(), &schema_mapping); window_expr.clone().with_new_expressions( - window_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, &schema_mapping)) - .collect(), + new_exprs, ) }) .collect::>>() @@ -3272,6 +3250,22 @@ fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } +fn update_sort_exprs(sort_exprs: &[PhysicalSortExpr], mapping: &HashMap) -> LexOrdering{ + sort_exprs + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, mapping), + options: sort_expr.options, + }) + .collect::>() +} +fn update_exprs(exprs: &[Arc], mapping: &HashMap) -> Vec> { + exprs + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect::>() +} + /// Given mapping representing the initial and new index values, /// it updates the indices of columns in the [`PhysicalExpr`]. fn update_column_index( From 60fea2715ce2646b3c3e1eb346fc5e47a14ce9ea Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 16:54:41 +0300 Subject: [PATCH 06/85] Simplifications --- .../optimize_projections.rs | 30 ++++--------------- 1 file changed, 5 insertions(+), 25 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 98257b303c7f..9d6f6a989353 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -2121,12 +2121,9 @@ impl ProjectionOptimizer { .zip(window_usage.clone()) .filter(|(_window_expr, (_window_col, usage))| *usage) .map(|(window_expr, (_window_col, _usage))| { + let new_exprs = update_exprs(&window_expr.expressions(), &schema_mapping); window_expr.clone().with_new_expressions( - window_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, &schema_mapping)) - .collect(), + new_exprs, ) }) .collect::>>() @@ -3492,10 +3489,7 @@ fn rewrite_repartition( mapping: &HashMap, ) -> Result> { let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { - let new_exprs = exprs - .iter() - .map(|expr| update_column_index(expr, &mapping)) - .collect::>(); + let new_exprs = update_exprs(exprs, &mapping); Partitioning::Hash(new_exprs, *size) } else { partitioning.clone() @@ -3508,14 +3502,7 @@ fn rewrite_sort( input_plan: Arc, mapping: &HashMap, ) -> Result> { - let new_sort_exprs = sort - .expr() - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, &mapping), - options: sort_expr.options, - }) - .collect::>(); + let new_sort_exprs = update_sort_exprs(sort.expr(), &mapping); Ok(Arc::new( SortExec::new(new_sort_exprs, input_plan) .with_fetch(sort.fetch()) @@ -3528,14 +3515,7 @@ fn rewrite_sort_preserving_merge( input_plan: Arc, mapping: &HashMap, ) -> Result> { - let new_sort_exprs = sort - .expr() - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, &mapping), - options: sort_expr.options, - }) - .collect::>(); + let new_sort_exprs = update_sort_exprs(sort.expr(), &mapping); Ok(Arc::new( SortPreservingMergeExec::new(new_sort_exprs, input_plan).with_fetch(sort.fetch()), ) as _) From e98326deed30240ed566b90d61a5e9c437a8f6c7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 17:21:48 +0300 Subject: [PATCH 07/85] Simpifications --- .../optimize_projections.rs | 179 +++--------------- 1 file changed, 30 insertions(+), 149 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9d6f6a989353..5f936c1e7cf3 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3532,52 +3532,15 @@ fn rewrite_hash_join( .on() .into_iter() .map(|(left, right)| { - let mut left_state = RewriteState::Unchanged; - let mut right_state = RewriteState::Unchanged; ( - left.clone() - .transform_up_mut(&mut |expr: Arc| { - if left_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - left_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new( - update_column_index( - &(Arc::new(column.clone()) as _), - &mapping, - ) - .as_any() - .downcast_ref::() - .unwrap() - .clone(), - ))) - }) - .unwrap(), - right - .clone() - .transform_up_mut(&mut |expr: Arc| { - if right_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - right_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new( - update_column_index( - &(Arc::new(column.clone()) as _), - &mapping, - ) - .as_any() - .downcast_ref::() - .unwrap() - .clone(), - ))) - }) - .unwrap(), + update_column_index( + &left, + &mapping, + ), + update_column_index( + &right, + &mapping, + ), ) }) .collect(); @@ -3684,52 +3647,15 @@ fn rewrite_sort_merge_join( .on() .into_iter() .map(|(left, right)| { - let mut left_state = RewriteState::Unchanged; - let mut right_state = RewriteState::Unchanged; ( - left.clone() - .transform_up_mut(&mut |expr: Arc| { - if left_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - left_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new( - update_column_index( - &(Arc::new(column.clone()) as _), - &mapping, - ) - .as_any() - .downcast_ref::() - .unwrap() - .clone(), - ))) - }) - .unwrap(), - right - .clone() - .transform_up_mut(&mut |expr: Arc| { - if right_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - right_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new( - update_column_index( - &(Arc::new(column.clone()) as _), - &mapping, - ) - .as_any() - .downcast_ref::() - .unwrap() - .clone(), - ))) - }) - .unwrap(), + update_column_index( + &left, + &mapping, + ), + update_column_index( + &right, + &mapping, + ), ) }) .collect(); @@ -3788,46 +3714,15 @@ fn rewrite_symmetric_hash_join( .on() .into_iter() .map(|(left, right)| { - let mut left_state = RewriteState::Unchanged; - let mut right_state = RewriteState::Unchanged; ( - left.clone() - .transform_up_mut(&mut |expr: Arc| { - if left_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - left_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new( - update_column_index(&(left.clone()), &mapping) - .as_any() - .downcast_ref::() - .unwrap() - .clone(), - ))) - }) - .unwrap(), - right - .clone() - .transform_up_mut(&mut |expr: Arc| { - if right_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); - }; - right_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new( - update_column_index(&(right.clone()), &mapping) - .as_any() - .downcast_ref::() - .unwrap() - .clone(), - ))) - }) - .unwrap(), + update_column_index( + &left, + &mapping, + ), + update_column_index( + &right, + &mapping, + ), ) }) .collect(); @@ -3943,12 +3838,9 @@ fn rewrite_window_aggregate( .window_expr() .iter() .map(|window_expr| { + let new_exprs = update_exprs(&window_expr.expressions(), mapping); window_expr.clone().with_new_expressions( - window_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(), + new_exprs, ) }) .collect::>>() @@ -3957,11 +3849,7 @@ fn rewrite_window_aggregate( } else { return Ok(None); }; - let new_partition_keys = w_agg - .partition_keys - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(); + let new_partition_keys = update_exprs(&w_agg.partition_keys, mapping); WindowAggExec::try_new(new_window, input_plan, new_partition_keys) .map(|plan| Some(Arc::new(plan) as _)) } @@ -3975,12 +3863,9 @@ fn rewrite_bounded_window_aggregate( .window_expr() .iter() .map(|window_expr| { + let new_exprs = update_exprs(&window_expr.expressions(), mapping); window_expr.clone().with_new_expressions( - window_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(), + new_exprs, ) }) .collect::>>() @@ -3989,11 +3874,7 @@ fn rewrite_bounded_window_aggregate( } else { return Ok(None); }; - let new_partition_keys = bw_agg - .partition_keys - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(); + let new_partition_keys = update_exprs(&bw_agg.partition_keys, mapping); BoundedWindowAggExec::try_new( new_window, input_plan, From 2af385a6b32e56df727552f0c6f0eba38d5f1243 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 17:38:54 +0300 Subject: [PATCH 08/85] Simplifications --- .../optimize_projections.rs | 193 ++++++++---------- 1 file changed, 85 insertions(+), 108 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 5f936c1e7cf3..dd51b15f6d60 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -69,12 +69,17 @@ use datafusion_common::DataFusionError; use datafusion_common::{internal_err, JoinSide, JoinType}; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::collect_columns; -use datafusion_physical_expr::{LexOrdering, Partitioning, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr::window::WindowExpr; +use datafusion_physical_expr::{ + AggregateExpr, LexOrdering, Partitioning, PhysicalExpr, PhysicalSortExpr, +}; use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::insert::FileSinkExec; -use datafusion_physical_plan::joins::utils::{ColumnIndex, JoinFilter, JoinOn}; +use datafusion_physical_plan::joins::utils::{ + ColumnIndex, JoinFilter, JoinOn, JoinOnRef, +}; use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, NestedLoopJoinExec, SortMergeJoinExec, SymmetricHashJoinExec, @@ -2031,10 +2036,9 @@ impl ProjectionOptimizer { .zip(window_usage.clone()) .filter(|(_window_expr, (_window_col, usage))| *usage) .map(|(window_expr, (_window_col, _usage))| { - let new_exprs = update_exprs(&window_expr.expressions(), &schema_mapping); - window_expr.clone().with_new_expressions( - new_exprs, - ) + let new_exprs = + update_exprs(&window_expr.expressions(), &schema_mapping); + window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() .unwrap(); @@ -2121,10 +2125,9 @@ impl ProjectionOptimizer { .zip(window_usage.clone()) .filter(|(_window_expr, (_window_col, usage))| *usage) .map(|(window_expr, (_window_col, _usage))| { - let new_exprs = update_exprs(&window_expr.expressions(), &schema_mapping); - window_expr.clone().with_new_expressions( - new_exprs, - ) + let new_exprs = + update_exprs(&window_expr.expressions(), &schema_mapping); + window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() .unwrap(); @@ -3247,7 +3250,10 @@ fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } -fn update_sort_exprs(sort_exprs: &[PhysicalSortExpr], mapping: &HashMap) -> LexOrdering{ +fn update_sort_exprs( + sort_exprs: &[PhysicalSortExpr], + mapping: &HashMap, +) -> LexOrdering { sort_exprs .iter() .map(|sort_expr| PhysicalSortExpr { @@ -3256,13 +3262,59 @@ fn update_sort_exprs(sort_exprs: &[PhysicalSortExpr], mapping: &HashMap>() } -fn update_exprs(exprs: &[Arc], mapping: &HashMap) -> Vec> { +fn update_exprs( + exprs: &[Arc], + mapping: &HashMap, +) -> Vec> { exprs .iter() .map(|expr| update_column_index(expr, mapping)) .collect::>() } +fn update_window_exprs( + window_exprs: &[Arc], + mapping: &HashMap, +) -> Option>> { + window_exprs + .iter() + .map(|window_expr| { + let new_exprs = update_exprs(&window_expr.expressions(), mapping); + window_expr.clone().with_new_expressions(new_exprs) + }) + .collect::>>() +} + +fn update_aggregate_exprs( + aggregate_exprs: &[Arc], + mapping: &HashMap, +) -> Option>> { + aggregate_exprs + .iter() + .map(|aggr_expr| { + aggr_expr.clone().with_new_expressions( + aggr_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) + }) + .collect::>>() +} + +fn update_join_on(join_on: JoinOnRef, mapping: &HashMap) -> JoinOn { + join_on + .into_iter() + .map(|(left, right)| { + ( + update_column_index(&left, mapping), + update_column_index(&right, mapping), + ) + }) + .collect() +} + /// Given mapping representing the initial and new index values, /// it updates the indices of columns in the [`PhysicalExpr`]. fn update_column_index( @@ -3528,22 +3580,7 @@ fn rewrite_hash_join( mapping: &HashMap, left_size: usize, ) -> Result> { - let new_on = hj - .on() - .into_iter() - .map(|(left, right)| { - ( - update_column_index( - &left, - &mapping, - ), - update_column_index( - &right, - &mapping, - ), - ) - }) - .collect(); + let new_on = update_join_on(hj.on(), mapping); let new_filter = hj.filter().map(|filter| { JoinFilter::new( filter.expression().clone(), @@ -3643,22 +3680,7 @@ fn rewrite_sort_merge_join( mapping: &HashMap, left_size: usize, ) -> Result> { - let new_on = smj - .on() - .into_iter() - .map(|(left, right)| { - ( - update_column_index( - &left, - &mapping, - ), - update_column_index( - &right, - &mapping, - ), - ) - }) - .collect(); + let new_on = update_join_on(smj.on(), mapping); let new_filter = smj.filter.as_ref().map(|filter| { JoinFilter::new( filter.expression().clone(), @@ -3710,22 +3732,7 @@ fn rewrite_symmetric_hash_join( mapping: &HashMap, left_size: usize, ) -> Result> { - let new_on = shj - .on() - .into_iter() - .map(|(left, right)| { - ( - update_column_index( - &left, - &mapping, - ), - update_column_index( - &right, - &mapping, - ), - ) - }) - .collect(); + let new_on = update_join_on(shj.on(), mapping); let new_filter = shj.filter().map(|filter| { JoinFilter::new( filter.expression().clone(), @@ -3791,24 +3798,12 @@ fn rewrite_aggregate( .collect(), agg.group_expr().groups().to_vec(), ); - let new_agg_expr = if let Some(new_agg_expr) = agg - .aggr_expr() - .iter() - .map(|aggr_expr| { - aggr_expr.clone().with_new_expressions( - aggr_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(), - ) - }) - .collect::>>() - { - new_agg_expr - } else { - return Ok(None); - }; + let new_agg_expr = + if let Some(new_agg_expr) = update_aggregate_exprs(agg.aggr_expr(), mapping) { + new_agg_expr + } else { + return Ok(None); + }; let new_filter = agg .filter_expr() .iter() @@ -3834,21 +3829,12 @@ fn rewrite_window_aggregate( input_plan: Arc, mapping: &HashMap, ) -> Result>> { - let new_window = if let Some(new_window) = w_agg - .window_expr() - .iter() - .map(|window_expr| { - let new_exprs = update_exprs(&window_expr.expressions(), mapping); - window_expr.clone().with_new_expressions( - new_exprs, - ) - }) - .collect::>>() - { - new_window - } else { - return Ok(None); - }; + let new_window = + if let Some(new_window) = update_window_exprs(&w_agg.window_expr(), mapping) { + new_window + } else { + return Ok(None); + }; let new_partition_keys = update_exprs(&w_agg.partition_keys, mapping); WindowAggExec::try_new(new_window, input_plan, new_partition_keys) .map(|plan| Some(Arc::new(plan) as _)) @@ -3859,21 +3845,12 @@ fn rewrite_bounded_window_aggregate( input_plan: Arc, mapping: &HashMap, ) -> Result>> { - let new_window = if let Some(new_window) = bw_agg - .window_expr() - .iter() - .map(|window_expr| { - let new_exprs = update_exprs(&window_expr.expressions(), mapping); - window_expr.clone().with_new_expressions( - new_exprs, - ) - }) - .collect::>>() - { - new_window - } else { - return Ok(None); - }; + let new_window = + if let Some(new_window) = update_window_exprs(&bw_agg.window_expr(), mapping) { + new_window + } else { + return Ok(None); + }; let new_partition_keys = update_exprs(&bw_agg.partition_keys, mapping); BoundedWindowAggExec::try_new( new_window, From 56b31363f55f02246dc485fcfbc016e2add77e7d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 16 Feb 2024 18:15:43 +0300 Subject: [PATCH 09/85] Minor changes --- .../physical_optimizer/optimize_projections.rs | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index dd51b15f6d60..7017851513a6 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -674,13 +674,7 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the hashed expressions if there is any with possibly updated column indices. - let new_partitioning = - if let Partitioning::Hash(exprs, size) = repartition.partitioning() { - let updated_exprs = update_exprs(exprs, &schema_mapping); - Partitioning::Hash(updated_exprs, *size) - } else { - repartition.partitioning().clone() - }; + let new_partitioning = update_partitioning(repartition.partitioning(), &schema_mapping); let plan = Arc::new(RepartitionExec::try_new( new_child.plan.clone(), new_partitioning, @@ -3272,6 +3266,15 @@ fn update_exprs( .collect::>() } +fn update_partitioning(partitioning: &Partitioning, mapping: &HashMap,) -> Partitioning{ + if let Partitioning::Hash(exprs, size) = partitioning { + let updated_exprs = update_exprs(exprs, mapping); + Partitioning::Hash(updated_exprs, *size) + } else { + partitioning.clone() + } +} + fn update_window_exprs( window_exprs: &[Arc], mapping: &HashMap, From 26542ba4f264ddbe697a0465fb6043feaecf04d0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 20 Feb 2024 14:10:19 +0300 Subject: [PATCH 10/85] tpch fails --- .../optimize_projections.rs | 668 +++++++++++++----- datafusion/physical-expr/src/aggregate/mod.rs | 2 +- .../physical-expr/src/window/window_expr.rs | 6 +- datafusion/physical-plan/src/projection.rs | 2 +- datafusion/sqllogictest/test_files/cte.slt | 3 +- .../sqllogictest/test_files/group_by.slt | 100 +-- datafusion/sqllogictest/test_files/joins.slt | 178 ++--- .../sqllogictest/test_files/tpch/q1.slt.part | 5 +- .../sqllogictest/test_files/tpch/q10.slt.part | 20 +- .../sqllogictest/test_files/tpch/q11.slt.part | 24 +- .../sqllogictest/test_files/tpch/q12.slt.part | 8 +- .../sqllogictest/test_files/tpch/q13.slt.part | 8 +- .../sqllogictest/test_files/tpch/q14.slt.part | 8 +- .../sqllogictest/test_files/tpch/q15.slt.part | 16 +- .../sqllogictest/test_files/tpch/q16.slt.part | 8 +- .../sqllogictest/test_files/tpch/q17.slt.part | 12 +- .../sqllogictest/test_files/tpch/q18.slt.part | 12 +- .../sqllogictest/test_files/tpch/q19.slt.part | 8 +- .../sqllogictest/test_files/tpch/q2.slt.part | 111 ++- .../sqllogictest/test_files/tpch/q20.slt.part | 26 +- .../sqllogictest/test_files/tpch/q21.slt.part | 33 +- .../sqllogictest/test_files/tpch/q22.slt.part | 9 +- .../sqllogictest/test_files/tpch/q3.slt.part | 17 +- .../sqllogictest/test_files/tpch/q4.slt.part | 14 +- .../sqllogictest/test_files/tpch/q5.slt.part | 29 +- .../sqllogictest/test_files/tpch/q6.slt.part | 4 +- .../sqllogictest/test_files/tpch/q7.slt.part | 95 ++- .../sqllogictest/test_files/tpch/q8.slt.part | 126 ++-- .../sqllogictest/test_files/tpch/q9.slt.part | 84 +-- datafusion/sqllogictest/test_files/union.slt | 35 +- 30 files changed, 1009 insertions(+), 662 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 7017851513a6..c3486303ea5e 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -62,7 +62,6 @@ use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::ExecutionPlan; use arrow_schema::SchemaRef; -use chrono::naive; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::DataFusionError; @@ -91,7 +90,7 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::union::{InterleaveExec, UnionExec}; use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion_physical_plan::{displayable, get_plan_string}; -use itertools::{Interleave, Itertools}; +use itertools::Itertools; /// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary /// fields for column requirements and changed indices of columns. @@ -145,14 +144,29 @@ impl ProjectionOptimizer { return Ok(self); } - if self.plan.as_any().is::() { + let x = if self.plan.as_any().is::() { // If the node is a projection, it is analyzed and may be rewritten // in a most effective way, or even removed. self.optimize_projections() } else { // If the node corresponds to any other plan, a projection may be inserted to its input. self.try_projection_insertion() - } + }?; + + // print_plan(&x.plan); + // println!("self reqs: {:?}", x.required_columns); + // println!("self map: {:?}", x.schema_mapping); + // x.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // x.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // x.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + + Ok(x) } /// The function tries 4 cases: @@ -237,7 +251,9 @@ impl ProjectionOptimizer { let mut projected_exprs = vec![]; for (expr, alias) in projection.expr() { - let Some(expr) = update_expr(expr, child_projection.expr(), true)? else { + let Some(expr) = + update_expr_with_projection(expr, child_projection.expr(), true)? + else { return Ok(Transformed::No(self)); }; projected_exprs.push((expr, alias.clone())); @@ -520,8 +536,9 @@ impl ProjectionOptimizer { self.children_nodes[0].required_columns = collect_columns_in_plan_schema(&self.children_nodes[0].plan) } else { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan) + }); return Ok(self); } Ok(self) @@ -674,7 +691,10 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the hashed expressions if there is any with possibly updated column indices. - let new_partitioning = update_partitioning(repartition.partitioning(), &schema_mapping); + let new_partitioning = update_partitioning_expressions( + repartition.partitioning(), + &schema_mapping, + ); let plan = Arc::new(RepartitionExec::try_new( new_child.plan.clone(), new_partitioning, @@ -705,7 +725,7 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. - let new_sort_exprs = update_sort_exprs(sort.expr(), &schema_mapping); + let new_sort_exprs = update_sort_expressions(sort.expr(), &schema_mapping); let plan = Arc::new( SortExec::new(new_sort_exprs, new_child.plan.clone()) .with_preserve_partitioning(sort.preserve_partitioning()) @@ -741,7 +761,8 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. - let new_sort_exprs = update_sort_exprs(sortp_merge.expr(), &schema_mapping); + let new_sort_exprs = + update_sort_expressions(sortp_merge.expr(), &schema_mapping); let plan = Arc::new( SortPreservingMergeExec::new(new_sort_exprs, new_child.plan.clone()) .with_fetch(sortp_merge.fetch()), @@ -819,7 +840,7 @@ impl ProjectionOptimizer { all_columns_required(&analyzed_join_right), ) { // We need two projections on top of both children. - (true, true) => { + (false, false) => { let (new_left_child, new_right_child, schema_mapping) = self .insert_multi_projections_below_join( left_size, @@ -839,7 +860,7 @@ impl ProjectionOptimizer { } } // Left child needs a projection. - (true, false) => { + (false, true) => { let right_child = self.children_nodes.swap_remove(1); let (new_left_child, left_schema_mapping) = self.insert_projection_below_single_child(analyzed_join_left, 0)?; @@ -856,7 +877,7 @@ impl ProjectionOptimizer { } } // Right child needs a projection. - (false, true) => { + (true, false) => { let left_child = self.children_nodes[0].clone(); let (new_right_child, mut right_schema_mapping) = self.insert_projection_below_single_child(analyzed_join_right, 1)?; @@ -882,7 +903,7 @@ impl ProjectionOptimizer { } } // All columns are required. - (false, false) => { + (true, true) => { self.required_columns = HashSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); @@ -2030,8 +2051,10 @@ impl ProjectionOptimizer { .zip(window_usage.clone()) .filter(|(_window_expr, (_window_col, usage))| *usage) .map(|(window_expr, (_window_col, _usage))| { - let new_exprs = - update_exprs(&window_expr.expressions(), &schema_mapping); + let new_exprs = update_expressions( + &window_expr.expressions(), + &schema_mapping, + ); window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() @@ -2119,8 +2142,10 @@ impl ProjectionOptimizer { .zip(window_usage.clone()) .filter(|(_window_expr, (_window_col, usage))| *usage) .map(|(window_expr, (_window_col, _usage))| { - let new_exprs = - update_exprs(&window_expr.expressions(), &schema_mapping); + let new_exprs = update_expressions( + &window_expr.expressions(), + &schema_mapping, + ); window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() @@ -2598,7 +2623,8 @@ impl ProjectionOptimizer { hj, self.children_nodes[0].plan.clone(), self.children_nodes[1].plan.clone(), - &new_mapping, + &left_mapping, + &right_mapping, left_size, )?; match hj.join_type() { @@ -2669,7 +2695,8 @@ impl ProjectionOptimizer { smj, self.children_nodes[0].plan.clone(), self.children_nodes[1].plan.clone(), - &new_mapping, + &left_mapping, + &right_mapping, left_size, )?; all_mappings[0] = match smj.join_type() { @@ -2699,7 +2726,8 @@ impl ProjectionOptimizer { shj, self.children_nodes[0].plan.clone(), self.children_nodes[1].plan.clone(), - &new_mapping, + &left_mapping, + &right_mapping, left_size, )?; all_mappings[0] = match shj.join_type() { @@ -2712,6 +2740,15 @@ impl ProjectionOptimizer { }; self.update_mapping(all_mappings) } else if let Some(agg) = plan_any.downcast_ref::() { + if agg.aggr_expr().iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .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(), @@ -2721,7 +2758,6 @@ impl ProjectionOptimizer { } else { return Ok(Transformed::No(self)); }; - self.update_mapping(all_mappings) } else if let Some(w_agg) = plan_any.downcast_ref::() { self.plan = if let Some(updated) = rewrite_window_aggregate( w_agg, @@ -2929,69 +2965,62 @@ impl TreeNode for ProjectionOptimizer { .map(transform) .collect::>>()?; + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + self = match self.index_updater()? { Transformed::Yes(updated) => updated, Transformed::No(not_rewritable) => { ProjectionOptimizer::new_default(not_rewritable.plan) } }; + + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); + // 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) - } - } -} -fn calculate_column_mapping( - required_columns: &HashSet, - unused_columns: &HashSet, -) -> HashMap { - let mut new_mapping = HashMap::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 -} + // print_plan(&self.plan); + // println!("self reqs: {:?}", self.required_columns); + // println!("self map: {:?}", self.schema_mapping); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // self.children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); -fn split_column_reqs(reqs: &ColumnRequirements) -> (HashSet, HashSet) { - let mut required = HashSet::new(); - let mut unused = HashSet::new(); - for (col, is_req) in reqs { - if *is_req { - required.insert(col.clone()); - } else { - unused.insert(col.clone()); + Ok(self) } } - (required, unused) -} - -fn convert_projection_exprs( - cols: HashSet, -) -> Vec<(Arc, String)> { - let mut result = cols - .into_iter() - .map(|col| { - let name = col.name().to_string(); - (Arc::new(col) as Arc, name) - }) - .collect::>(); - result } #[derive(Default)] @@ -3015,6 +3044,7 @@ impl PhysicalOptimizerRule for OptimizeProjections { plan: Arc, _config: &ConfigOptions, ) -> Result> { + print_plan(&plan); // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); let mut optimizer = ProjectionOptimizer::new_default(plan); @@ -3023,8 +3053,39 @@ impl PhysicalOptimizerRule for OptimizeProjections { let mut optimized = optimizer.transform_down(&|o| { o.adjust_node_with_requirements().map(Transformed::Yes) })?; + // print_plan(&optimized.plan); + // println!("self reqs: {:?}", optimized.required_columns); + // println!("self map: {:?}", optimized.schema_mapping); + // optimized + // .children_nodes + // .iter() + // .for_each(|c: &ProjectionOptimizer| { + // print_plan(&c.plan); + // }); + // optimized + // .children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // optimized + // .children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); // Ensure the final optimized plan satisfies the initial schema requirements. optimized = satisfy_initial_schema(optimized, initial_requirements)?; + // print_plan(&optimized.plan); + // println!("self reqs: {:?}", optimized.required_columns); + // println!("self map: {:?}", optimized.schema_mapping); + // optimized.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // }); + // optimized + // .children_nodes + // .iter() + // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); + // optimized + // .children_nodes + // .iter() + // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); // TODO: Remove this check to tests crosscheck_helper(optimized.clone())?; @@ -3099,6 +3160,53 @@ fn satisfy_initial_schema( } } +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: HashMap::new(), // Reset schema mapping as we've now satisfied the initial schema + children_nodes: vec![po], // Keep the original node as the child + }) + } +} + /// Iterates over all columns and returns true if all columns are required. fn all_columns_required(requirement_map: &ColumnRequirements) -> bool { requirement_map.iter().all(|(_k, v)| *v) @@ -3171,6 +3279,62 @@ fn new_projections_for_columns( .collect() } +/// 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: &HashSet, + unused_columns: &HashSet, +) -> HashMap { + let mut new_mapping = HashMap::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 `ColumnRequirements`, it separates the required and redundant columns. +fn split_column_reqs(reqs: &ColumnRequirements) -> (HashSet, HashSet) { + let mut required = HashSet::new(); + let mut unused = HashSet::new(); + for (col, is_req) in reqs { + if *is_req { + required.insert(col.clone()); + } else { + unused.insert(col.clone()); + } + } + (required, unused) +} + +/// 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( + cols: HashSet, +) -> Vec<(Arc, String)> { + let result = cols + .into_iter() + .map(|col| { + let name = col.name().to_string(); + (Arc::new(col) as Arc, name) + }) + .collect::>(); + result +} + #[derive(Debug, PartialEq)] enum RewriteState { /// The expression is unchanged. @@ -3199,7 +3363,7 @@ enum RewriteState { /// 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( +fn update_expr_with_projection( expr: &Arc, projected_exprs: &[(Arc, String)], sync_with_child: bool, @@ -3244,7 +3408,19 @@ fn update_expr( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) } -fn update_sort_exprs( +/// Rewrites the expressions with new index values. +fn update_expressions( + exprs: &[Arc], + mapping: &HashMap, +) -> Vec> { + exprs + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect::>() +} + +/// Rewrites the sort expressions with new index values. +fn update_sort_expressions( sort_exprs: &[PhysicalSortExpr], mapping: &HashMap, ) -> LexOrdering { @@ -3256,25 +3432,22 @@ fn update_sort_exprs( }) .collect::>() } -fn update_exprs( - exprs: &[Arc], - mapping: &HashMap, -) -> Vec> { - exprs - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect::>() -} -fn update_partitioning(partitioning: &Partitioning, mapping: &HashMap,) -> Partitioning{ +/// Updates the expressions subject to hashing of the `Partitioning` according to +/// the mapping. If it is not a hash partitioning, they remains as they are. +fn update_partitioning_expressions( + partitioning: &Partitioning, + mapping: &HashMap, +) -> Partitioning { if let Partitioning::Hash(exprs, size) = partitioning { - let updated_exprs = update_exprs(exprs, mapping); + let updated_exprs = update_expressions(exprs, mapping); Partitioning::Hash(updated_exprs, *size) } else { partitioning.clone() } } +/// Rewrites the window expressions with new index values. fn update_window_exprs( window_exprs: &[Arc], mapping: &HashMap, @@ -3282,12 +3455,13 @@ fn update_window_exprs( window_exprs .iter() .map(|window_expr| { - let new_exprs = update_exprs(&window_expr.expressions(), mapping); + let new_exprs = update_expressions(&window_expr.expressions(), mapping); window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() } +/// Rewrites the aggregate expressions with new index values. fn update_aggregate_exprs( aggregate_exprs: &[Arc], mapping: &HashMap, @@ -3306,13 +3480,18 @@ fn update_aggregate_exprs( .collect::>>() } -fn update_join_on(join_on: JoinOnRef, mapping: &HashMap) -> JoinOn { +/// Rewrites the expressions in equivalence condition of a join with new index values. +fn update_join_on( + join_on: JoinOnRef, + left_mapping: &HashMap, + right_mapping: &HashMap, +) -> JoinOn { join_on .into_iter() .map(|(left, right)| { ( - update_column_index(&left, mapping), - update_column_index(&right, mapping), + update_column_index(&left, left_mapping), + update_column_index(&right, right_mapping), ) }) .collect() @@ -3544,7 +3723,7 @@ fn rewrite_repartition( mapping: &HashMap, ) -> Result> { let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { - let new_exprs = update_exprs(exprs, &mapping); + let new_exprs = update_expressions(exprs, &mapping); Partitioning::Hash(new_exprs, *size) } else { partitioning.clone() @@ -3557,7 +3736,7 @@ fn rewrite_sort( input_plan: Arc, mapping: &HashMap, ) -> Result> { - let new_sort_exprs = update_sort_exprs(sort.expr(), &mapping); + let new_sort_exprs = update_sort_expressions(sort.expr(), &mapping); Ok(Arc::new( SortExec::new(new_sort_exprs, input_plan) .with_fetch(sort.fetch()) @@ -3570,7 +3749,7 @@ fn rewrite_sort_preserving_merge( input_plan: Arc, mapping: &HashMap, ) -> Result> { - let new_sort_exprs = update_sort_exprs(sort.expr(), &mapping); + 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 _) @@ -3580,10 +3759,11 @@ fn rewrite_hash_join( hj: &HashJoinExec, left_input_plan: Arc, right_input_plan: Arc, - mapping: &HashMap, + left_mapping: &HashMap, + right_mapping: &HashMap, left_size: usize, ) -> Result> { - let new_on = update_join_on(hj.on(), mapping); + let new_on = update_join_on(hj.on(), left_mapping, right_mapping); let new_filter = hj.filter().map(|filter| { JoinFilter::new( filter.expression().clone(), @@ -3592,7 +3772,7 @@ fn rewrite_hash_join( .iter() .map(|col_idx| match col_idx.side { JoinSide::Left => ColumnIndex { - index: mapping + index: left_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index @@ -3602,7 +3782,7 @@ fn rewrite_hash_join( side: JoinSide::Left, }, JoinSide::Right => ColumnIndex { - index: mapping + index: right_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index + left_size @@ -3680,10 +3860,11 @@ fn rewrite_sort_merge_join( smj: &SortMergeJoinExec, left_input_plan: Arc, right_input_plan: Arc, - mapping: &HashMap, + left_mapping: &HashMap, + right_mapping: &HashMap, left_size: usize, ) -> Result> { - let new_on = update_join_on(smj.on(), mapping); + 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(), @@ -3692,7 +3873,7 @@ fn rewrite_sort_merge_join( .iter() .map(|col_idx| match col_idx.side { JoinSide::Left => ColumnIndex { - index: mapping + index: left_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index @@ -3702,7 +3883,7 @@ fn rewrite_sort_merge_join( side: JoinSide::Left, }, JoinSide::Right => ColumnIndex { - index: mapping + index: right_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index + left_size @@ -3732,10 +3913,11 @@ fn rewrite_symmetric_hash_join( shj: &SymmetricHashJoinExec, left_input_plan: Arc, right_input_plan: Arc, - mapping: &HashMap, + left_mapping: &HashMap, + right_mapping: &HashMap, left_size: usize, ) -> Result> { - let new_on = update_join_on(shj.on(), mapping); + let new_on = update_join_on(shj.on(), left_mapping, right_mapping); let new_filter = shj.filter().map(|filter| { JoinFilter::new( filter.expression().clone(), @@ -3744,7 +3926,7 @@ fn rewrite_symmetric_hash_join( .iter() .map(|col_idx| match col_idx.side { JoinSide::Left => ColumnIndex { - index: mapping + index: left_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index @@ -3754,7 +3936,7 @@ fn rewrite_symmetric_hash_join( side: JoinSide::Left, }, JoinSide::Right => ColumnIndex { - index: mapping + index: right_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index + left_size @@ -3838,7 +4020,7 @@ fn rewrite_window_aggregate( } else { return Ok(None); }; - let new_partition_keys = update_exprs(&w_agg.partition_keys, mapping); + let new_partition_keys = update_expressions(&w_agg.partition_keys, mapping); WindowAggExec::try_new(new_window, input_plan, new_partition_keys) .map(|plan| Some(Arc::new(plan) as _)) } @@ -3849,12 +4031,12 @@ fn rewrite_bounded_window_aggregate( mapping: &HashMap, ) -> Result>> { let new_window = - if let Some(new_window) = update_window_exprs(&bw_agg.window_expr(), mapping) { + if let Some(new_window) = update_window_exprs(bw_agg.window_expr(), mapping) { new_window } else { return Ok(None); }; - let new_partition_keys = update_exprs(&bw_agg.partition_keys, mapping); + let new_partition_keys = update_expressions(&bw_agg.partition_keys, mapping); BoundedWindowAggExec::try_new( new_window, input_plan, @@ -3873,7 +4055,7 @@ mod tests { use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::execution::context::SessionContext; use crate::physical_optimizer::optimize_projections::{ - update_expr, OptimizeProjections, + update_expr_with_projection, OptimizeProjections, }; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; @@ -4086,7 +4268,7 @@ mod tests { )?), ]; for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &child, true)? + assert!(update_expr_with_projection(&expr, &child, true)? .unwrap() .eq(&expected_expr)); } @@ -4220,7 +4402,7 @@ mod tests { )?), ]; for (expr, expected_expr) in exprs.into_iter().zip(expected_exprs.into_iter()) { - assert!(update_expr(&expr, &projected_exprs, false)? + assert!(update_expr_with_projection(&expr, &projected_exprs, false)? .unwrap() .eq(&expected_expr)); } @@ -4746,76 +4928,242 @@ mod tests { Ok(()) } + // #[tokio::test] + // async fn test_trivial() -> Result<()> { + // let config = SessionConfig::new() + // .with_target_partitions(2) + // .with_batch_size(4096); + // let ctx = SessionContext::with_config(config); + // let _dataframe = ctx + // .sql( + // "CREATE EXTERNAL TABLE aggregate_test_100 ( + // c1 VARCHAR NOT NULL, + // c2 TINYINT NOT NULL, + // c3 SMALLINT NOT NULL, + // c4 SMALLINT, + // c5 INT, + // c6 BIGINT NOT NULL, + // c7 SMALLINT NOT NULL, + // c8 INT NOT NULL, + // c9 BIGINT UNSIGNED NOT NULL, + // c10 VARCHAR NOT NULL, + // c11 FLOAT NOT NULL, + // c12 DOUBLE NOT NULL, + // c13 VARCHAR NOT NULL + // ) + // STORED AS CSV + // WITH HEADER ROW + // LOCATION '/Users/berkaysahin/Desktop/datafusion-upstream/datafusion/core/tests/tpch-csv/lineitem.csv'", + // ) + // .await?; + + // let dataframe = ctx + // .sql( + // "WITH indices AS ( + // SELECT 1 AS idx UNION ALL + // SELECT 2 AS idx UNION ALL + // SELECT 3 AS idx UNION ALL + // SELECT 4 AS idx UNION ALL + // SELECT 5 AS idx + // ) + // SELECT data.arr[indices.idx] as element, array_length(data.arr) as array_len, dummy + // FROM ( + // SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100 + // ) data + // CROSS JOIN indices + // ORDER BY 1", + // ) + // .await?; + // let physical_plan = dataframe.clone().create_physical_plan().await?; + // let batches = dataframe.collect().await?; + // let _ = print_plan(&physical_plan); + // let _ = print_batches(&batches); + // Ok(()) + // } + + /// TODO: Result of this query is wrong, where output schema orders is different what is should be. + /// The problem originates in `AggregateExec: mode=FinalPartitioned` at the top. #[tokio::test] - async fn test_trivial() -> Result<()> { - let mut config = SessionConfig::new() - .with_target_partitions(2) - .with_batch_size(4096); + async fn test_trivial2() -> Result<()> { + let mut config = SessionConfig::new(); let ctx = SessionContext::with_config(config); let _dataframe = ctx .sql( - "CREATE EXTERNAL TABLE aggregate_test_100 ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 BIGINT UNSIGNED NOT NULL, - c10 VARCHAR NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL - ) - STORED AS CSV - WITH HEADER ROW - LOCATION '../../testing/data/csv/aggregate_test_100.csv'", + "CREATE TABLE web_sales ( + ws_sold_date_sk INT NOT NULL, + ws_sold_time_sk INT NOT NULL, + ws_ship_date_sk INT NOT NULL, + ws_item_sk INT NOT NULL, + ws_bill_customer_sk INT NOT NULL, + ws_bill_cdemo_sk INT NOT NULL, + ws_bill_hdemo_sk INT NOT NULL, + ws_bill_addr_sk INT NOT NULL, + ws_ship_customer_sk INT NOT NULL, + ws_ship_cdemo_sk INT NOT NULL, + ws_ship_hdemo_sk INT NOT NULL, + ws_ship_addr_sk INT NOT NULL, + ws_web_page_sk INT NOT NULL, + ws_web_site_sk INT NOT NULL, + ws_ship_mode_sk INT NOT NULL, + ws_warehouse_sk INT NOT NULL, + ws_promo_sk INT NOT NULL, + ws_order_number BIGINT NOT NULL, + ws_quantity INT NOT NULL, + ws_wholesale_cost DECIMAL(10,2) NOT NULL, + ws_list_price DECIMAL(10,2) NOT NULL, + ws_sales_price DECIMAL(10,2) NOT NULL, + ws_ext_discount_amt DECIMAL(10,2) NOT NULL, + ws_ext_sales_price DECIMAL(10,2) NOT NULL, + ws_ext_wholesale_cost DECIMAL(10,2) NOT NULL, + ws_ext_list_price DECIMAL(10,2) NOT NULL, + ws_ext_tax DECIMAL(10,2) NOT NULL, + ws_coupon_amt DECIMAL(10,2) NOT NULL, + ws_ext_ship_cost DECIMAL(10,2) NOT NULL, + ws_net_paid DECIMAL(10,2) NOT NULL, + ws_net_paid_inc_tax DECIMAL(10,2) NOT NULL, + ws_net_paid_inc_ship DECIMAL(10,2) NOT NULL, + ws_net_paid_inc_ship_tax DECIMAL(10,2) NOT NULL, + ws_net_profit DECIMAL(10,2) NOT NULL +);", ) .await?; - let dataframe = ctx + let _dataframe = ctx .sql( - "WITH indices AS ( - SELECT 1 AS idx UNION ALL - SELECT 2 AS idx UNION ALL - SELECT 3 AS idx UNION ALL - SELECT 4 AS idx UNION ALL - SELECT 5 AS idx -) -SELECT data.arr[indices.idx] as element, array_length(data.arr) as array_len, dummy -FROM ( - SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100 -) data - CROSS JOIN indices -ORDER BY 1", + "CREATE TABLE catalog_sales ( + cs_sold_date_sk INT NOT NULL, + cs_sold_time_sk INT NOT NULL, + cs_ship_date_sk INT NOT NULL, + cs_bill_customer_sk INT NOT NULL, + cs_bill_cdemo_sk INT NOT NULL, + cs_bill_hdemo_sk INT NOT NULL, + cs_bill_addr_sk INT NOT NULL, + cs_ship_customer_sk INT NOT NULL, + cs_ship_cdemo_sk INT NOT NULL, + cs_ship_hdemo_sk INT NOT NULL, + cs_ship_addr_sk INT NOT NULL, + cs_call_center_sk INT NOT NULL, + cs_catalog_page_sk INT NOT NULL, + cs_ship_mode_sk INT NOT NULL, + cs_warehouse_sk INT NOT NULL, + cs_item_sk INT NOT NULL, + cs_promo_sk INT NOT NULL, + cs_order_number BIGINT NOT NULL, + cs_quantity INT NOT NULL, + cs_wholesale_cost DECIMAL(10,2) NOT NULL, + cs_list_price DECIMAL(10,2) NOT NULL, + cs_sales_price DECIMAL(10,2) NOT NULL, + cs_ext_discount_amt DECIMAL(10,2) NOT NULL, + cs_ext_sales_price DECIMAL(10,2) NOT NULL, + cs_ext_wholesale_cost DECIMAL(10,2) NOT NULL, + cs_ext_list_price DECIMAL(10,2) NOT NULL, + cs_ext_tax DECIMAL(10,2) NOT NULL, + cs_coupon_amt DECIMAL(10,2) NOT NULL, + cs_ext_ship_cost DECIMAL(10,2) NOT NULL, + cs_net_paid DECIMAL(10,2) NOT NULL, + cs_net_paid_inc_tax DECIMAL(10,2) NOT NULL, + cs_net_paid_inc_ship DECIMAL(10,2) NOT NULL, + cs_net_paid_inc_ship_tax DECIMAL(10,2) NOT NULL, + cs_net_profit DECIMAL(10,2) NOT NULL +);", ) .await?; - let physical_plan = dataframe.clone().create_physical_plan().await?; - let batches = dataframe.collect().await?; - let _ = print_plan(&physical_plan); - let _ = print_batches(&batches); - Ok(()) - } - /// TODO: Result of this query is wrong, where output schema orders is different what is should be. - /// The problem originates in `AggregateExec: mode=FinalPartitioned` at the top. - #[tokio::test] - async fn test_trivial2() -> Result<()> { - let mut config = SessionConfig::new() - .with_target_partitions(2) - .with_batch_size(4096); - let ctx = SessionContext::with_config(config); let _dataframe = ctx .sql( - "CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER) as VALUES (83,0,38), (26,0,79), (43,81,24)", + "CREATE TABLE date_dim ( + d_date_sk INT NOT NULL, + d_date_id VARCHAR NOT NULL, + d_date DATE NOT NULL, + d_month_seq INT NOT NULL, + d_week_seq INT NOT NULL, + d_quarter_seq INT NOT NULL, + d_year INT NOT NULL, + d_dow INT NOT NULL, + d_moy INT NOT NULL, + d_dom INT NOT NULL, + d_qoy INT NOT NULL, + d_fy_year INT NOT NULL, + d_fy_quarter_seq INT NOT NULL, + d_fy_week_seq INT NOT NULL, + d_day_name VARCHAR NOT NULL, + d_quarter_name VARCHAR NOT NULL, + d_holiday VARCHAR, + d_weekend VARCHAR, + d_following_holiday VARCHAR, + d_first_dom INT NOT NULL, + d_last_dom INT NOT NULL, + d_same_day_ly INT NOT NULL, + d_same_day_lq INT NOT NULL, + d_current_day VARCHAR, + d_current_week VARCHAR, + d_current_month VARCHAR, + d_current_quarter VARCHAR, + d_current_year VARCHAR +); +", ) .await?; let dataframe = ctx .sql( - "SELECT DISTINCT * FROM tab0 AS cor0 GROUP BY cor0.col1, cor0.col2, cor0.col0", + "with wscs as + (select sold_date_sk + ,sales_price + from (select ws_sold_date_sk sold_date_sk + ,ws_ext_sales_price sales_price + from web_sales + union all + select cs_sold_date_sk sold_date_sk + ,cs_ext_sales_price sales_price + from catalog_sales)), + wswscs as + (select d_week_seq, + sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, + sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, + sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, + sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, + sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, + sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, + sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales + from wscs + ,date_dim + where d_date_sk = sold_date_sk + group by d_week_seq) + select d_week_seq1 + ,round(sun_sales1/sun_sales2,2) + ,round(mon_sales1/mon_sales2,2) + ,round(tue_sales1/tue_sales2,2) + ,round(wed_sales1/wed_sales2,2) + ,round(thu_sales1/thu_sales2,2) + ,round(fri_sales1/fri_sales2,2) + ,round(sat_sales1/sat_sales2,2) + from + (select wswscs.d_week_seq d_week_seq1 + ,sun_sales sun_sales1 + ,mon_sales mon_sales1 + ,tue_sales tue_sales1 + ,wed_sales wed_sales1 + ,thu_sales thu_sales1 + ,fri_sales fri_sales1 + ,sat_sales sat_sales1 + from wswscs,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2001) y, + (select wswscs.d_week_seq d_week_seq2 + ,sun_sales sun_sales2 + ,mon_sales mon_sales2 + ,tue_sales tue_sales2 + ,wed_sales wed_sales2 + ,thu_sales thu_sales2 + ,fri_sales fri_sales2 + ,sat_sales sat_sales2 + from wswscs + ,date_dim + where date_dim.d_week_seq = wswscs.d_week_seq and + d_year = 2001+1) z + where d_week_seq1=d_week_seq2-53 + order by d_week_seq1;", ) .await?; @@ -4823,7 +5171,7 @@ ORDER BY 1", let batches = dataframe.collect().await?; let _ = print_plan(&physical_plan); let _ = print_batches(&batches); - assert!(false); + Ok(()) } } diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index c4a1aa7fbb5f..e111721f81cf 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -138,7 +138,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// The order of the given expressions is taken into account while replacing. fn with_new_expressions( self: Arc, - expressions: Vec>, + _expressions: Vec>, ) -> Option> { None } diff --git a/datafusion/physical-expr/src/window/window_expr.rs b/datafusion/physical-expr/src/window/window_expr.rs index 78b653a47b7e..3a3d088b0799 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -27,9 +27,7 @@ use arrow::compute::kernels::sort::SortColumn; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; -use datafusion_common::{ - internal_err, not_impl_err, DataFusionError, Result, ScalarValue, -}; +use datafusion_common::{internal_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::window_state::{ PartitionBatchState, WindowAggState, WindowFrameContext, }; @@ -134,7 +132,7 @@ pub trait WindowExpr: Send + Sync + Debug { /// The order of the given expressions is taken into account while replacing. fn with_new_expressions( self: Arc, - expressions: Vec>, + _expressions: Vec>, ) -> Option> { None } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 6c3d21955607..25f5b5f354ae 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -74,7 +74,7 @@ impl ProjectionExec { expr = expr .iter() .enumerate() - .map(|(expr_idx, (expression, name))| { + .map(|(_expr_idx, (expression, name))| { expression .clone() .transform_down(&|e| match e.as_any().downcast_ref::() { diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 172db0bc0570..6b9db5589391 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -137,8 +137,7 @@ Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.accou physical_plan SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST] --RecursiveQueryExec: name=balances, is_distinct=false -----ProjectionExec: expr=[time@0 as time, name@1 as name, account_balance@2 as account_balance] -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true ----CoalescePartitionsExec ------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] --------CoalesceBatchesExec: target_batch_size=2 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 213a3f6b52ec..54da1dd22355 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2015,7 +2015,7 @@ Sort: l.col0 ASC NULLS LAST physical_plan SortPreservingMergeExec: [col0@0 ASC NULLS LAST] --SortExec: expr=[col0@0 ASC NULLS LAST] -----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] +----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1)@3 as last_col1] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 @@ -2144,7 +2144,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotate --Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan -ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] +ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c)@2 as summation1] --AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) ----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] @@ -2177,7 +2177,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE( --Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] +ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c)@2 as first_c] --AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -2203,7 +2203,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a --Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] +ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] --AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -2463,7 +2463,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales --Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2493,7 +2493,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] ----SubqueryAlias: s ------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2537,7 +2537,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2573,7 +2573,7 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[zip_code, country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] +ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount)@2 as amounts, SUM(s.amount)@3 as sum1] --AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2609,7 +2609,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2644,7 +2644,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2676,7 +2676,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales --Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts, FIRST_VALUE(sales_global.amount)@2 as fv1, LAST_VALUE(sales_global.amount)@3 as fv2] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2707,7 +2707,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales --Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts, FIRST_VALUE(sales_global.amount)@2 as fv1, LAST_VALUE(sales_global.amount)@3 as fv2] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2739,7 +2739,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal --Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] +ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as fv2, ARRAY_AGG(sales_global.amount)@3 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2769,7 +2769,7 @@ Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_globa --Aggregate: groupBy=[[sales_global.country]], aggr=[[SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] +ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount)@1 as sum1, ARRAY_AGG(sales_global.amount)@2 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@2 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2804,7 +2804,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ----Sort: sales_global.ts ASC NULLS LAST ------TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +ProjectionExec: expr=[country@0 as country, LAST_VALUE(sales_global.amount)@1 as fv1, FIRST_VALUE(sales_global.amount)@2 as lv1, SUM(sales_global.amount)@3 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] ----MemoryExec: partitions=1, partition_sizes=[1] @@ -2837,7 +2837,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal --Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] ----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as lv1, SUM(sales_global.amount)@3 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] ----MemoryExec: partitions=1, partition_sizes=[1] @@ -2873,13 +2873,14 @@ Sort: s.sn ASC NULLS LAST ------------TableScan: sales_global projection=[sn, ts, currency, amount] physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] ---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] +--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)@5 as last_rate] ----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)] -------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, amount@3 as amount] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] +--------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount] +----------CoalesceBatchesExec: target_batch_size=8192 +------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2918,7 +2919,7 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] +----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as fv2] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 @@ -2954,7 +2955,7 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] +----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as fv2] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 @@ -2990,7 +2991,7 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS --Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] +ProjectionExec: expr=[FIRST_VALUE(sales_global.amount)@0 as fv1, LAST_VALUE(sales_global.amount)@1 as fv2] --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] @@ -3016,7 +3017,7 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS --Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] +ProjectionExec: expr=[FIRST_VALUE(sales_global.amount)@0 as fv1, LAST_VALUE(sales_global.amount)@1 as fv2] --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] @@ -3040,7 +3041,7 @@ Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS L --Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as array_agg1] +ProjectionExec: expr=[ARRAY_AGG(sales_global.amount)@0 as array_agg1] --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] @@ -3064,7 +3065,7 @@ Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS --Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@0 as array_agg1] +ProjectionExec: expr=[ARRAY_AGG(sales_global.amount)@0 as array_agg1] --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] @@ -3088,7 +3089,7 @@ Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NUL --Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@0 as array_agg1] +ProjectionExec: expr=[ARRAY_AGG(sales_global.amount)@0 as array_agg1] --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] @@ -3117,7 +3118,7 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] +----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as array_agg1] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 @@ -3153,7 +3154,7 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts, FIRST_VALUE(sales_global.amount)@2 as fv1, LAST_VALUE(sales_global.amount)@3 as fv2] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 @@ -3420,9 +3421,9 @@ Sort: r.sn ASC NULLS LAST ----------SubqueryAlias: r ------------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -SortPreservingMergeExec: [sn@0 ASC NULLS LAST] ---SortExec: expr=[sn@0 ASC NULLS LAST] -----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] +ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] +--SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +----SortExec: expr=[sn@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 @@ -3568,9 +3569,9 @@ Sort: l.sn ASC NULLS LAST ------------SubqueryAlias: l --------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] physical_plan -SortPreservingMergeExec: [sn@2 ASC NULLS LAST] ---SortExec: expr=[sn@2 ASC NULLS LAST] -----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] +ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] +--SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +----SortExec: expr=[sn@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 @@ -3799,7 +3800,7 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab --Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a)@1 as first_a, FIRST_VALUE(multiple_ordered_table.c)@2 as last_c] --AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 @@ -3869,10 +3870,10 @@ Projection: amount_usd ----------------SubqueryAlias: r ------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +ProjectionExec: expr=[LAST_VALUE(l.d)@1 as amount_usd] --AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted -----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] -------CoalesceBatchesExec: target_batch_size=2 +----CoalesceBatchesExec: target_batch_size=2 +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] --------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true ----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] @@ -4027,15 +4028,16 @@ Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 --------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))]] ----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, c@3 as c, sum1@2 as sum1, sum1@5 as sum1] +ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)] -------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true -------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +----ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, c@3 as c, sum1@5 as sum1] +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)] +--------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +----------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +--------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +----------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 9619696679d2..ef4d02216142 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1325,8 +1325,8 @@ Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] ------TableScan: join_t2 projection=[t2_id] physical_plan AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] ---ProjectionExec: expr=[t1_id@0 as t1_id] -----CoalesceBatchesExec: target_batch_size=2 +--CoalesceBatchesExec: target_batch_size=2 +----ProjectionExec: expr=[t1_id@0 as t1_id] ------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] --------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 @@ -1353,8 +1353,8 @@ Projection: COUNT(*) physical_plan ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] --AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] -----ProjectionExec: expr=[t1_id@0 as t1_id] -------CoalesceBatchesExec: target_batch_size=2 +----CoalesceBatchesExec: target_batch_size=2 +------ProjectionExec: expr=[t1_id@0 as t1_id] --------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] ----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 @@ -1386,8 +1386,8 @@ ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] ------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] --------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] ----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -------------ProjectionExec: expr=[t1_id@0 as t1_id] ---------------CoalesceBatchesExec: target_batch_size=2 +------------CoalesceBatchesExec: target_batch_size=2 +--------------ProjectionExec: expr=[t1_id@0 as t1_id] ----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] ------------------CoalesceBatchesExec: target_batch_size=2 --------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 @@ -1454,16 +1454,17 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ ----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] +--------CoalescePartitionsExec +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1480,19 +1481,20 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ ----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] +------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 +------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 +------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1511,16 +1513,17 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] +--------CoalescePartitionsExec +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1537,19 +1540,20 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] +------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 +------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 +------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1571,11 +1575,12 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] -------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@1 as t1_id, t1_name@2 as t1_name] +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] +--------MemoryExec: partitions=1, partition_sizes=[1] +--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1595,16 +1600,17 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@1 as t1_id, t1_name@2 as t1_name] +------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 +------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1624,15 +1630,16 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] +--------CoalescePartitionsExec +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1650,18 +1657,19 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ----TableScan: join_t1 projection=[t1_id, t1_name] ----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] +------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 +------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1680,8 +1688,8 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) --TableScan: join_t1 projection=[t1_id, t1_name, t1_int] --TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] ---CoalesceBatchesExec: target_batch_size=2 +CoalesceBatchesExec: target_batch_size=2 +--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] ----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] ------MemoryExec: partitions=1, partition_sizes=[1] ------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] @@ -1703,8 +1711,8 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) --TableScan: join_t1 projection=[t1_id, t1_name, t1_int] --TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] ---CoalesceBatchesExec: target_batch_size=2 +CoalesceBatchesExec: target_batch_size=2 +--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] ----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] ------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 @@ -2038,8 +2046,8 @@ NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------MemoryExec: partitions=1, partition_sizes=[1] --CoalescePartitionsExec -----ProjectionExec: expr=[t2_id@0 as t2_id] -------CoalesceBatchesExec: target_batch_size=2 +----CoalesceBatchesExec: target_batch_size=2 +------ProjectionExec: expr=[t2_id@0 as t2_id] --------FilterExec: t2_int@1 > 1 ----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3364,7 +3372,7 @@ Sort: l.a ASC NULLS FIRST --------SubqueryAlias: r ----------TableScan: annotated_data projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b)@3 as last_col1] --AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] @@ -3412,10 +3420,10 @@ Projection: amount_usd ----------------SubqueryAlias: r ------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +ProjectionExec: expr=[LAST_VALUE(l.d)@1 as amount_usd] --AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted -----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] -------CoalesceBatchesExec: target_batch_size=2 +----CoalesceBatchesExec: target_batch_size=2 +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] --------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true ----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] @@ -3450,7 +3458,7 @@ Sort: l.a ASC NULLS FIRST physical_plan SortPreservingMergeExec: [a@0 ASC] --SortExec: expr=[a@0 ASC] -----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b)@3 as last_col1] ------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] --------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 3086ab487aaa..dcd9dd3abe58 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -56,8 +56,9 @@ SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS ------------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(*)] --------------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_discountlineitem.l_discountDecimal128(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] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------FilterExec: l_shipdate@6 <= 10471 ---------------------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 +------------------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] +--------------------FilterExec: l_shipdate@6 <= 10471 +----------------------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 eb0b66f024de..f118e9b4b0f8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -78,18 +78,18 @@ GlobalLimitExec: skip=0, fetch=10 ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] -------------------CoalesceBatchesExec: target_batch_size=8192 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] --------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)] ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] -----------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] ------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)] --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] ----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 @@ -97,14 +97,14 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------------------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 ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] --------------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 ----------------------------------------------------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 --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------------------------------------FilterExec: l_returnflag@3 = R ------------------------------------------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 ----------------------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 4efa29e2c0ac..42c766403e3a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -83,13 +83,13 @@ GlobalLimitExec: skip=0, fetch=10 ------------CoalesceBatchesExec: target_batch_size=8192 --------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] ----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] --------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 @@ -100,8 +100,8 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] --------------------------------FilterExec: n_name@1 = GERMANY ----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false @@ -109,13 +109,13 @@ GlobalLimitExec: skip=0, fetch=10 ------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] --------------CoalescePartitionsExec ----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------------ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] ----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] --------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 @@ -126,8 +126,8 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] --------------------------------FilterExec: n_name@1 = GERMANY ----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------------------------------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 68ef41b382e3..a4eb58ec6752 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -66,13 +66,13 @@ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] ---------------ProjectionExec: expr=[l_shipmode@1 as l_shipmode, o_orderpriority@3 as o_orderpriority] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[l_shipmode@1 as l_shipmode, o_orderpriority@3 as o_orderpriority] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] ----------------------------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 ------------------------------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 --------------------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 5cf6ace8b27b..19aeaa5894aa 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -64,8 +64,8 @@ GlobalLimitExec: skip=0, fetch=10 --------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] ----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] ------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] ---------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] -----------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] ------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 @@ -73,8 +73,8 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false --------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ----------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% ------------------------------------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 7e614ab49e38..155005210d23 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -45,13 +45,13 @@ ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") --AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_type@4 as p_type] -----------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_type@4 as p_type] ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 ------------------------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 --------------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 a872e96acf04..6bc9c5f87760 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -73,13 +73,13 @@ Sort: supplier.s_suppkey ASC NULLS LAST physical_plan SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] --SortExec: expr=[s_suppkey@0 ASC NULLS LAST] -----ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue] -------CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=8192 +------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue] --------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)] ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 ---------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 @@ -90,8 +90,8 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 ----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ------------------------------------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 ----------CoalesceBatchesExec: target_batch_size=8192 @@ -104,8 +104,8 @@ SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 ----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ------------------------------------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 b93872929fe5..080a3592a5d6 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -82,8 +82,8 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] ------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)] --------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 @@ -96,8 +96,8 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------------------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 ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] ------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% --------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------------------------------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 4d4aa4b1395f..600e7e5fa78b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -55,19 +55,19 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av --AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] ---------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] -----------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 ---------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ------------------------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 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------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 53191a5d44e1..5cacf104ee06 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -75,13 +75,13 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] ------------CoalesceBatchesExec: target_batch_size=8192 --------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] -----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] -------------------CoalesceBatchesExec: target_batch_size=8192 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] --------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)] ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] -----------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] ------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 @@ -93,8 +93,8 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 --------------------------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 -----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -------------------CoalesceBatchesExec: target_batch_size=8192 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] --------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 ----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] ------------------------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 2df27bd41082..859b2f63e958 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -68,13 +68,13 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco --AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -----------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] ------------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 --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -------------------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] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------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] ----------------------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 ------------------------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 --------------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 ed950db190bb..aa33680eedf7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -100,76 +100,76 @@ Limit: skip=0, fetch=10 --------------------Filter: region.r_name = Utf8("EUROPE") ----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 -----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] -------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@8 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] +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] +--GlobalLimitExec: skip=0, fetch=10 +----SortPreservingMergeExec: [s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST], fetch=10 +------SortExec: TopK(fetch=10), expr=[s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@1), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@0)] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 -----------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, ps_supplycost@7 as ps_supplycost, n_name@8 as n_name] +----------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, n_name@8 as n_name] +------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@1)] +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@5 as s_phone, s_acctbal@6 as s_acctbal, s_comment@7 as s_comment, ps_supplycost@8 as ps_supplycost, n_name@10 as n_name, n_regionkey@11 as n_regionkey] +--------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, ps_supplycost@7 as ps_supplycost, n_name@8 as n_name] +----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)] +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 ----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment, ps_supplycost@3 as ps_supplycost] +------------------------------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_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost, n_name@10 as n_name, n_regionkey@11 as n_regionkey] +--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@5, n_nationkey@0)] +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 --------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -----------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] +----------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@3 as ps_supplycost, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment] +------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)] +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 ------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +--------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] +----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)] +------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS ---------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------------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 -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ---------------------------------------------------------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 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------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 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------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 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +--------------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS +----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------------------------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 +------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +----------------------------------------------------------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 +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------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 +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------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 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: r_name@1 = EUROPE ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([ps_partkey@1, MIN(partsupp.ps_supplycost)@0], 4), input_partitions=4 -----------------ProjectionExec: expr=[MIN(partsupp.ps_supplycost)@1 as MIN(partsupp.ps_supplycost), ps_partkey@0 as ps_partkey] +------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +--------------------------------FilterExec: r_name@1 = EUROPE +----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 ------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] ---------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] -----------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] ------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)] --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] ----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] ------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] --------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 @@ -184,14 +184,13 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ----------------------------------------FilterExec: r_name@1 = EUROPE ------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - query RTTITTTT select s_acctbal, diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index e014c6cafd98..85fcbf12b01d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -83,13 +83,13 @@ Sort: supplier.s_name ASC NULLS LAST physical_plan SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] --SortExec: expr=[s_name@0 ASC NULLS LAST] -----ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] -------CoalesceBatchesExec: target_batch_size=8192 +----CoalesceBatchesExec: target_batch_size=8192 +------ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] --------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)] ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 @@ -97,15 +97,15 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] --------------------------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 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] ----------------------------FilterExec: n_name@1 = CANADA ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 @@ -116,8 +116,8 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] --------------------------------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 ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ------------------------------------FilterExec: p_name@1 LIKE forest% --------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false @@ -126,13 +126,11 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 ----------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] ----------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 ------------------------------------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 - - query TT select s_name, diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index 147afc603c2c..57f548fd885f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -97,25 +97,25 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] ---------------ProjectionExec: expr=[s_name@0 as s_name] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[s_name@0 as s_name] ------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] --------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 ---------------------------------------ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] ------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)] --------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -------------------------------------------------ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] ----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] ------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 @@ -123,20 +123,20 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ------------------------------------------------------------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 ------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -----------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] --------------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 ----------------------------------------------------------------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 --------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] ----------------------------------------------------FilterExec: o_orderstatus@1 = F ------------------------------------------------------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 ----------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] ------------------------------------------FilterExec: n_name@1 = SAUDI ARABIA --------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false @@ -145,13 +145,12 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ----------------------------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 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] ----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 ------------------------------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 - query TI select s_name, diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index 2713d5bf6e18..3a650c2f3574 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -83,8 +83,8 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] --------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] ----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 -------------------ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] ---------------------CoalesceBatchesExec: target_batch_size=8192 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] ----------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)] ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 @@ -98,13 +98,12 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] --------------------CoalescePartitionsExec ----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] -------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] ----------------------------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") }]) ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------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 select cntrycode, diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 85f2d9986c27..939dc4d30ef2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -67,18 +67,18 @@ GlobalLimitExec: skip=0, fetch=10 ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] -------------------CoalesceBatchesExec: target_batch_size=8192 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] --------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)] ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] -----------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] ------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] --------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] ----------------------------------------FilterExec: c_mktsegment@1 = BUILDING ------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false @@ -89,13 +89,12 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------------------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 ----------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ------------------------------FilterExec: l_shipdate@3 > 9204 --------------------------------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 - query IRDI select l_orderkey, diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index 1709ae04aa3b..3f229670428a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -60,24 +60,22 @@ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] ---------------ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] ------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] ----------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 ------------------------------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 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] ----------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 ------------------------------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 - - query TI select o_orderpriority, diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index af3a33497026..11c31335b544 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -74,28 +74,28 @@ SortPreservingMergeExec: [revenue@1 DESC] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] -----------------CoalesceBatchesExec: target_batch_size=8192 +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] ----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] ------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] --------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)] ----------------------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 ---------------------------------------------ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] ------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)] --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -------------------------------------------------------ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] ----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 @@ -103,8 +103,8 @@ SortPreservingMergeExec: [revenue@1 DESC] ------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] --------------------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 ----------------------------------------------------------------------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 --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 @@ -120,14 +120,13 @@ SortPreservingMergeExec: [revenue@1 DESC] ------------------------------------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 --------------------CoalesceBatchesExec: target_batch_size=8192 ----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ---------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ----------------------------FilterExec: r_name@1 = ASIA ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - query TR select n_name, diff --git a/datafusion/sqllogictest/test_files/tpch/q6.slt.part b/datafusion/sqllogictest/test_files/tpch/q6.slt.part index 8e53be297db0..983ee5082453 100644 --- a/datafusion/sqllogictest/test_files/tpch/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q6.slt.part @@ -38,8 +38,8 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as r --AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] ---------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------CoalesceBatchesExec: target_batch_size=8192 +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ------------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 --------------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/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 5186c46a896f..7f912f36aeb4 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -91,59 +91,58 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] ---------------ProjectionExec: expr=[n_name@4 as supp_nation, n_name@6 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] +--------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@6 as n_name] +------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_shipdate@2 as l_shipdate, n_name@4 as n_name, n_name@6 as n_name] +--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 --------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 -----------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] +----------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@6 as n_name] +------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)] +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)] -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 ---------------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] +--------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] +----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)] +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 ----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)] ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -------------------------------------------------------ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] +------------------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] +--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)] +----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 --------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 ---------------------------------------------------------------------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_extendedprice, l_discount, l_shipdate], has_header=false ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------------------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], has_header=false -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - +----------------------------------------------------------ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] +------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] +--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 +----------------------------------------------------------------------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_extendedprice, l_discount, l_shipdate], has_header=false +----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +--------------------------------------------------------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], has_header=false +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query TTRR select diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 760b40ad1ae8..7d7f94a183e0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -97,78 +97,78 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] ---------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@4 as nation] +--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] +------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@2 as o_orderdate, n_name@4 as n_name] +--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 --------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] +----------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] +------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)] -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 ---------------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] +--------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] +----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)] +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 ----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)] ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 -------------------------------------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] +------------------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] +--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)] +----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 --------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] +----------------------------------------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] +------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] +--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)] -----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ---------------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] +--------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] +----------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)] +------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ----------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] ---------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +------------------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] +--------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] +----------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 --------------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL -------------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false ---------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -------------------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 ---------------------------------------------------------------------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 ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +----------------------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +------------------------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL +--------------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +----------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +--------------------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 +----------------------------------------------------------------------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 +----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 --------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: r_name@1 = AMERICA -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - +----------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +------------------------------FilterExec: r_name@1 = AMERICA +--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false query RR diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 5db97f79bdb1..8da8131a24d1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -84,54 +84,54 @@ GlobalLimitExec: skip=0, fetch=10 ----------CoalesceBatchesExec: target_batch_size=8192 ------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -----------------ProjectionExec: expr=[n_name@7 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@4 * l_quantity@0 as amount] +----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] +--------------------ProjectionExec: expr=[l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, ps_supplycost@4 as ps_supplycost, o_orderdate@5 as o_orderdate, n_name@7 as n_name] +----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] +------------------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] +--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 --------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 -----------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] +----------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] +------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)] +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 ------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 ---------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] +--------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] +----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)] +------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 ----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] ---------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] +--------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] +----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 --------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------FilterExec: p_name@1 LIKE %green% -------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false ---------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -------------------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -----------------------------------------------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 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------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], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - +----------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +------------------------------------------------------------------------FilterExec: p_name@1 LIKE %green% +--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +--------------------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +------------------------------------------------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 +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +--------------------------------------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], has_header=false +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query TRR diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 5550fa68a6fc..17ff7ff08d99 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -278,7 +278,7 @@ Union physical_plan UnionExec --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@2), (name@1, name@1)] +----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@1), (name@1, name@0)] ------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] --------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 @@ -286,26 +286,27 @@ UnionExec --------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] ------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@1, name@0], 4), input_partitions=4 +----------ProjectionExec: expr=[name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] ------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] --ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] ----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] ---------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[id@0 as id, name@1 as name] +--------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] +----------CoalesceBatchesExec: target_batch_size=2 +------------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +--------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +----------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +------------------CoalesceBatchesExec: target_batch_size=2 +--------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +----------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------MemoryExec: partitions=1, partition_sizes=[1] +----------CoalesceBatchesExec: target_batch_size=2 +------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort From 10000fb5bdb88a5e7a5cbfb81f3bd5d028bbeea7 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Thu, 22 Feb 2024 09:56:23 +0300 Subject: [PATCH 11/85] Delete docs.yaml --- .github/workflows/docs.yaml | 64 ------------------------------------- 1 file changed, 64 deletions(-) delete mode 100644 .github/workflows/docs.yaml diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml deleted file mode 100644 index ab6a615ab60b..000000000000 --- a/.github/workflows/docs.yaml +++ /dev/null @@ -1,64 +0,0 @@ -on: - push: - branches: - - main - paths: - - .asf.yaml - - .github/workflows/docs.yaml - - docs/** - -name: Deploy DataFusion site - -jobs: - build-docs: - name: Build docs - runs-on: ubuntu-latest - steps: - - name: Checkout docs sources - uses: actions/checkout@v4 - - - name: Checkout asf-site branch - uses: actions/checkout@v4 - with: - ref: asf-site - path: asf-site - - - name: Setup Python - uses: actions/setup-python@v5 - with: - python-version: "3.10" - - - name: Install dependencies - run: | - set -x - python3 -m venv venv - source venv/bin/activate - pip install -r docs/requirements.txt - - - name: Build docs - run: | - set -x - source venv/bin/activate - cd docs - ./build.sh - - - name: Copy & push the generated HTML - run: | - set -x - cd asf-site/ - rsync \ - -a \ - --delete \ - --exclude '/.git/' \ - ../docs/build/html/ \ - ./ - cp ../.asf.yaml . - touch .nojekyll - git status --porcelain - if [ "$(git status --porcelain)" != "" ]; then - git config user.name "github-actions[bot]" - git config user.email "github-actions[bot]@users.noreply.github.com" - git add --all - git commit -m 'Publish built docs triggered by ${{ github.sha }}' - git push || git push --force - fi From 04143abbf5bf1a6e90e97c3f2e4094cc4fa4aec1 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 22 Feb 2024 13:09:46 +0300 Subject: [PATCH 12/85] All tests pass, ready to be cleaned-up --- .../optimize_projections.rs | 570 +++++++----------- datafusion/physical-expr/src/aggregate/sum.rs | 17 +- .../src/expressions/get_indexed_field.rs | 18 +- .../physical-plan/src/aggregates/mod.rs | 2 +- .../sqllogictest/test_files/aggregate.slt | 13 +- .../sqllogictest/test_files/subquery.slt | 73 +-- 6 files changed, 278 insertions(+), 415 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index c3486303ea5e..35ac85d00b40 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -70,11 +70,13 @@ 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, LexOrdering, Partitioning, PhysicalExpr, PhysicalSortExpr, + AggregateExpr, LexOrdering, Partitioning, PhysicalExpr, PhysicalExprRef, + PhysicalSortExpr, }; use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::get_plan_string; use datafusion_physical_plan::insert::FileSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, @@ -89,7 +91,7 @@ use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::union::{InterleaveExec, UnionExec}; use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; -use datafusion_physical_plan::{displayable, get_plan_string}; + use itertools::Itertools; /// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary @@ -125,7 +127,7 @@ impl ProjectionOptimizer { /// 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 { + fn adjust_node_with_requirements(self) -> Result { // print_plan(&self.plan); // println!("self reqs: {:?}", self.required_columns); // println!("self map: {:?}", self.schema_mapping); @@ -140,7 +142,7 @@ impl ProjectionOptimizer { // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); // If the node is a source provdider, no need a change. - if self.children_nodes.len() == 0 { + if self.children_nodes.is_empty() { return Ok(self); } @@ -226,7 +228,9 @@ impl ProjectionOptimizer { "\"optimize_projections\" subrule must be used on ProjectionExec's." ); }; + // If there is nothing that could be better, insert the child requirements and continue. + self.children_nodes[0].required_columns = self .required_columns .iter() @@ -304,7 +308,7 @@ impl ProjectionOptimizer { let input_columns = collect_columns_in_plan_schema(projection_exec.input()); if input_columns .iter() - .all(|input_column| projection_requires.contains(&input_column)) + .all(|input_column| projection_requires.contains(input_column)) { let new_mapping = self .required_columns @@ -417,8 +421,13 @@ impl ProjectionOptimizer { .iter() .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) .collect::>(); - let new_projections = - new_projections_for_columns(&projection_columns, &file_scan.projection); + + let new_projections = new_projections_for_columns( + &projection_columns, + &file_scan + .projection + .unwrap_or((0..csv.schema().fields().len()).collect()), + ); file_scan.projection = Some(new_projections); @@ -459,7 +468,11 @@ impl ProjectionOptimizer { // These plans preserve the input schema, and do not add new requirements. else if let Some(coal_b) = plan.as_any().downcast_ref::() { self = self.try_insert_below_coalesce_batches(coal_b)?; - } else if let Some(_) = plan.as_any().downcast_ref::() { + } else if plan + .as_any() + .downcast_ref::() + .is_some() + { self = self.try_insert_below_coalesce_partitions()?; } else if let Some(glimit) = plan.as_any().downcast_ref::() { self = self.try_insert_below_global_limit(glimit)?; @@ -480,9 +493,9 @@ impl ProjectionOptimizer { self = self.try_insert_below_sort_preserving_merge(sortp_merge)?; } // Preserves schema and do not change requirements, but have multi-child. - else if let Some(_) = plan.as_any().downcast_ref::() { + else if plan.as_any().downcast_ref::().is_some() { self = self.try_insert_below_union()?; - } else if let Some(_) = plan.as_any().downcast_ref::() { + } else if plan.as_any().downcast_ref::().is_some() { self = self.try_insert_below_interleave()?; } // Concatenates schemas and do not change requirements. @@ -532,7 +545,7 @@ impl ProjectionOptimizer { return Ok(self); } self = self.try_insert_below_bounded_window_aggregate(bw_agg)? - } else if let Some(file_sink) = plan.as_any().downcast_ref::() { + } else if let Some(_file_sink) = plan.as_any().downcast_ref::() { self.children_nodes[0].required_columns = collect_columns_in_plan_schema(&self.children_nodes[0].plan) } else { @@ -555,6 +568,7 @@ impl ProjectionOptimizer { mem::take(&mut self.required_columns); } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let plan = Arc::new(CoalesceBatchesExec::new( new_child.plan.clone(), coal_batches.target_batch_size(), @@ -681,7 +695,7 @@ impl ProjectionOptimizer { // the requirements with the columns in the hashed expressions. if let Partitioning::Hash(exprs, _size) = repartition.partitioning() { self.required_columns - .extend(exprs.iter().flat_map(|expr| collect_columns(expr))); + .extend(exprs.iter().flat_map(collect_columns)); } let requirement_map = self.analyze_requirements(); @@ -861,9 +875,16 @@ impl ProjectionOptimizer { } // Left child needs a projection. (false, true) => { - let right_child = self.children_nodes.swap_remove(1); - let (new_left_child, left_schema_mapping) = - self.insert_projection_below_single_child(analyzed_join_left, 0)?; + let mut right_child = self.children_nodes.swap_remove(1); + let (new_left_child, left_schema_mapping) = self + .clone() + .insert_projection_below_single_child(analyzed_join_left, 0)?; + right_child.required_columns = self + .required_columns + .iter() + .filter(|col| col.index() >= left_size) + .map(|col| Column::new(col.name(), col.index() - left_size)) + .collect(); let plan = Arc::new(CrossJoinExec::new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -878,9 +899,10 @@ impl ProjectionOptimizer { } // Right child needs a projection. (true, false) => { - let left_child = self.children_nodes[0].clone(); - let (new_right_child, mut right_schema_mapping) = - self.insert_projection_below_single_child(analyzed_join_right, 1)?; + let mut left_child = self.children_nodes[0].clone(); + let (new_right_child, mut right_schema_mapping) = self + .clone() + .insert_projection_below_single_child(analyzed_join_right, 1)?; right_schema_mapping = right_schema_mapping .into_iter() .map(|(old, new)| { @@ -890,6 +912,12 @@ impl ProjectionOptimizer { ) }) .collect(); + left_child.required_columns = self + .required_columns + .iter() + .filter(|col| col.index() < left_size) + .cloned() + .collect(); let plan = Arc::new(CrossJoinExec::new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -964,6 +992,7 @@ impl ProjectionOptimizer { analyzed_join_left, analyzed_join_right, )?; + let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), new_right_child.plan.clone(), @@ -982,7 +1011,7 @@ impl ProjectionOptimizer { } } (false, true) => { - let right_child = self.children_nodes.swap_remove(1); + let mut right_child = self.children_nodes.swap_remove(1); let new_on = update_equivalence_conditions( hj.on(), &analyzed_join_left, @@ -993,11 +1022,17 @@ impl ProjectionOptimizer { &analyzed_join_right, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( + let (new_left_child, mut left_schema_mapping) = + self.clone().insert_projection_below_single_child( analyzed_join_left, 0, )?; + right_child.required_columns = self + .required_columns + .iter() + .filter(|col| col.index() >= left_size) + .map(|col| Column::new(col.name(), col.index() - left_size)) + .collect(); let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -1007,6 +1042,22 @@ impl ProjectionOptimizer { *hj.partition_mode(), hj.null_equals_null(), )?) as _; + let new_left_size = new_left_child.plan.schema().fields().len(); + 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::>(), + ); self = ProjectionOptimizer { plan, @@ -1016,7 +1067,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes.swap_remove(1); + let mut left_child = self.children_nodes[0].clone(); let new_on = update_equivalence_conditions( hj.on(), &HashMap::new(), @@ -1027,11 +1078,27 @@ impl ProjectionOptimizer { &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( + let (new_right_child, mut right_schema_mapping) = + self.clone().insert_projection_below_single_child( analyzed_join_right, 1, )?; + right_schema_mapping = right_schema_mapping + .into_iter() + .map(|(old, new)| { + ( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + left_size), + ) + }) + .collect(); + left_child.required_columns = self + .required_columns + .iter() + .filter(|col| col.index() < left_size) + .cloned() + .collect(); + let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1073,11 +1140,12 @@ impl ProjectionOptimizer { &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( + let (new_left_child, left_schema_mapping) = + self.clone().insert_projection_below_single_child( analyzed_join_left, 0, )?; + let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -1116,7 +1184,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes.swap_remove(0); + let mut left_child = self.children_nodes[0].clone(); let new_on = update_equivalence_conditions( hj.on(), &HashMap::new(), @@ -1133,6 +1201,7 @@ impl ProjectionOptimizer { analyzed_join_right, 1, )?; + let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1223,23 +1292,45 @@ impl ProjectionOptimizer { } } (false, true) => { - let right_child = self.children_nodes.swap_remove(1); + let mut right_child = self.children_nodes.swap_remove(1); let new_filter = update_non_equivalence_conditions( nlj.filter(), &analyzed_join_right, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( + let (new_left_child, mut left_schema_mapping) = + self.clone().insert_projection_below_single_child( analyzed_join_left, 0, )?; + right_child.required_columns = self + .required_columns + .iter() + .filter(|col| col.index() >= left_size) + .map(|col| Column::new(col.name(), col.index() - left_size)) + .collect(); let plan = Arc::new(NestedLoopJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), new_filter, nlj.join_type(), )?) as _; + let new_left_size = new_left_child.plan.schema().fields().len(); + 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::>(), + ); self = ProjectionOptimizer { plan, @@ -1249,17 +1340,32 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes.swap_remove(1); + let mut left_child = self.children_nodes[0].clone(); let new_filter = update_non_equivalence_conditions( nlj.filter(), &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( + let (new_right_child, mut right_schema_mapping) = + self.clone().insert_projection_below_single_child( analyzed_join_right, 1, )?; + right_schema_mapping = right_schema_mapping + .into_iter() + .map(|(old, new)| { + ( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + left_size), + ) + }) + .collect(); + left_child.required_columns = self + .required_columns + .iter() + .filter(|col| col.index() < left_size) + .cloned() + .collect(); let plan = Arc::new(NestedLoopJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1332,7 +1438,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes.swap_remove(0); + let mut left_child = self.children_nodes[0].clone(); let new_filter = update_non_equivalence_conditions( nlj.filter(), &HashMap::new(), @@ -1472,7 +1578,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes.swap_remove(1); + let left_child = self.children_nodes[0].clone(); let new_on = update_equivalence_conditions( smj.on(), &HashMap::new(), @@ -1571,7 +1677,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes.swap_remove(0); + let mut left_child = self.children_nodes[0].clone(); let new_on = update_equivalence_conditions( smj.on(), &HashMap::new(), @@ -1725,7 +1831,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes.swap_remove(1); + let left_child = self.children_nodes[0].clone(); let new_on = update_equivalence_conditions( shj.on(), &HashMap::new(), @@ -1828,7 +1934,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes.swap_remove(0); + let mut left_child = self.children_nodes[0].clone(); let new_on = update_equivalence_conditions( shj.on(), &HashMap::new(), @@ -1907,7 +2013,7 @@ impl ProjectionOptimizer { if !unused_aggr_exprs.is_empty() { let new_plan = AggregateExec::try_new( - agg.mode().clone(), + *agg.mode(), agg.group_expr().clone(), agg.aggr_expr() .iter() @@ -1923,21 +2029,15 @@ impl ProjectionOptimizer { .group_expr() .expr() .iter() - .map(|(e, alias)| collect_columns(e)) - .flatten() + .flat_map(|(e, _alias)| collect_columns(e)) .collect(); self.children_nodes[0].required_columns.extend( - new_plan - .aggr_expr() - .iter() - .map(|e| { - e.expressions() - .iter() - .map(|e| collect_columns(e)) - .flatten() - .collect::>() - }) - .flatten(), + 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 = HashSet::new(); @@ -1976,8 +2076,8 @@ impl ProjectionOptimizer { self.children_nodes[0].required_columns.extend( aggr_columns .into_iter() - .chain(group_columns.into_iter()) - .chain(filter_columns.into_iter()), + .chain(group_columns) + .chain(filter_columns), ) } _ => { @@ -2006,8 +2106,8 @@ impl ProjectionOptimizer { self.children_nodes[0].required_columns.extend( aggr_columns .into_iter() - .chain(group_columns.into_iter()) - .chain(filter_columns.into_iter()), + .chain(group_columns) + .chain(filter_columns), ); } }; @@ -2026,15 +2126,11 @@ impl ProjectionOptimizer { window_expr .expressions() .iter() - .flat_map(|expr| collect_columns(&expr)) + .flat_map(collect_columns) .collect::>() })); - self.required_columns.extend( - w_agg - .partition_keys - .iter() - .flat_map(|key| collect_columns(key)), - ); + self.required_columns + .extend(w_agg.partition_keys.iter().flat_map(collect_columns)); let requirement_map = self.analyze_requirements(); if !all_columns_required(&requirement_map) { if window_agg_required( @@ -2117,15 +2213,11 @@ impl ProjectionOptimizer { window_expr .expressions() .iter() - .flat_map(|expr| collect_columns(&expr)) + .flat_map(collect_columns) .collect::>() })); - self.required_columns.extend( - bw_agg - .partition_keys - .iter() - .flat_map(|key| collect_columns(key)), - ); + self.required_columns + .extend(bw_agg.partition_keys.iter().flat_map(collect_columns)); let requirement_map = self.analyze_requirements(); if !all_columns_required(&requirement_map) { if window_agg_required( @@ -2357,6 +2449,7 @@ impl ProjectionOptimizer { let new_mapping = calculate_column_mapping(&required_columns, &unused_columns); let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { plan: inserted_projection, required_columns, @@ -2637,7 +2730,7 @@ impl ProjectionOptimizer { col_initial.index() < left_size }); all_mappings.push(new_left); - all_mappings.push(new_right); + all_mappings[0].extend(new_right); } JoinType::LeftSemi | JoinType::LeftAnti => { all_mappings.push(left_mapping) @@ -2646,6 +2739,7 @@ impl ProjectionOptimizer { all_mappings.push(right_mapping) } }; + // self.schema_mapping = all_mappings; self.update_mapping(all_mappings) } else if let Some(nlj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); @@ -2665,16 +2759,28 @@ impl ProjectionOptimizer { nlj, self.children_nodes[0].plan.clone(), self.children_nodes[1].plan.clone(), - &new_mapping, + &left_mapping, + &right_mapping, left_size, )?; - all_mappings[0] = match nlj.join_type() { + match nlj.join_type() { JoinType::Right | JoinType::Full | JoinType::Left - | JoinType::Inner => new_mapping, - JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, - JoinType::RightAnti | JoinType::RightSemi => right_mapping, + | 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) + } }; self.update_mapping(all_mappings) } else if let Some(smj) = plan_any.downcast_ref::() { @@ -2780,7 +2886,7 @@ impl ProjectionOptimizer { return Ok(Transformed::No(self)); }; self.update_mapping(all_mappings) - } else if let Some(file_sink) = plan_any.downcast_ref::() { + } 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 = @@ -2789,7 +2895,7 @@ impl ProjectionOptimizer { .collect_vec(); existing_columns.sort_by_key(|col| col.index()); let mut exprs = vec![]; - for idx in 0..existing_columns.len() { + for (idx, _) in existing_columns.iter().enumerate() { if let Some((initial, _final)) = mapped_exprs .iter() .find(|(initial, _final)| initial.index() == idx) @@ -2844,7 +2950,7 @@ impl ProjectionOptimizer { .map(|(initial, new)| { ( initial.clone(), - child_map.get(&new).cloned().unwrap_or(new.clone()), + child_map.get(new).cloned().unwrap_or(new.clone()), ) }) .collect() @@ -2920,7 +3026,7 @@ impl ProjectionOptimizer { } } } - return Ok(self); + Ok(self) } } @@ -3032,19 +3138,14 @@ impl OptimizeProjections { Self {} } } -fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) -} + impl PhysicalOptimizerRule for OptimizeProjections { fn optimize( &self, plan: Arc, _config: &ConfigOptions, ) -> Result> { - print_plan(&plan); + // print_plan(&plan); // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); let mut optimizer = ProjectionOptimizer::new_default(plan); @@ -3071,6 +3172,7 @@ impl PhysicalOptimizerRule for OptimizeProjections { // .iter() // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); // Ensure the final optimized plan satisfies the initial schema requirements. + optimized = satisfy_initial_schema(optimized, initial_requirements)?; // print_plan(&optimized.plan); // println!("self reqs: {:?}", optimized.required_columns); @@ -3139,7 +3241,7 @@ fn satisfy_initial_schema( .iter() .map(|col| { // If there is a change, get the new index. - let column_index = po.schema_mapping.get(&col).unwrap_or(&col).index(); + let column_index = po.schema_mapping.get(col).unwrap_or(col).index(); let new_col = Arc::new(Column::new(col.name(), column_index)) as Arc; (new_col, col.name().to_string()) @@ -3269,14 +3371,8 @@ fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { /// Updates a source provider's projected columns according to the given /// projection operator's expressions. To use this function safely, one must /// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns( - projection: &[&Column], - source: &Option>, -) -> Vec { - projection - .iter() - .filter_map(|col| source.as_ref().map(|proj| proj[col.index()])) - .collect() +fn new_projections_for_columns(projection: &[&Column], source: &[usize]) -> Vec { + projection.iter().map(|col| source[col.index()]).collect() } /// When a field in a schema is decided to be redundant and planned to be dropped @@ -3487,11 +3583,11 @@ fn update_join_on( right_mapping: &HashMap, ) -> JoinOn { join_on - .into_iter() + .iter() .map(|(left, right)| { ( - update_column_index(&left, left_mapping), - update_column_index(&right, right_mapping), + update_column_index(left, left_mapping), + update_column_index(right, right_mapping), ) }) .collect() @@ -3538,7 +3634,7 @@ fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet, Arc)], + on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, left_size: usize, join_left_schema: SchemaRef, @@ -3549,7 +3645,6 @@ fn collect_columns_in_join_conditions( .flat_map(|(col_left, col_right)| { let left_columns = collect_columns(col_left); let right_columns = collect_columns(col_right); - let mut state = RewriteState::Unchanged; let right_columns = right_columns .into_iter() .map(|col| Column::new(col.name(), col.index() + left_size)) @@ -3577,13 +3672,13 @@ fn collect_columns_in_join_conditions( .unwrap_or_default(); equivalence_columns .into_iter() - .chain(non_equivalence_columns.into_iter()) + .chain(non_equivalence_columns) .collect() } /// Updates the equivalence conditions of the joins according to the new indices of columns. fn update_equivalence_conditions( - on: &[(Arc, Arc)], + on: &[(PhysicalExprRef, PhysicalExprRef)], requirement_map_left: &ColumnRequirements, requirement_map_right: &ColumnRequirements, ) -> JoinOn { @@ -3723,7 +3818,7 @@ fn rewrite_repartition( mapping: &HashMap, ) -> Result> { let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { - let new_exprs = update_expressions(exprs, &mapping); + let new_exprs = update_expressions(exprs, mapping); Partitioning::Hash(new_exprs, *size) } else { partitioning.clone() @@ -3736,7 +3831,7 @@ fn rewrite_sort( input_plan: Arc, mapping: &HashMap, ) -> Result> { - let new_sort_exprs = update_sort_expressions(sort.expr(), &mapping); + let new_sort_exprs = update_sort_expressions(sort.expr(), mapping); Ok(Arc::new( SortExec::new(new_sort_exprs, input_plan) .with_fetch(sort.fetch()) @@ -3749,7 +3844,7 @@ fn rewrite_sort_preserving_merge( input_plan: Arc, mapping: &HashMap, ) -> Result> { - let new_sort_exprs = update_sort_expressions(sort.expr(), &mapping); + 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 _) @@ -3812,7 +3907,8 @@ fn rewrite_nested_loop_join( nlj: &NestedLoopJoinExec, left_input_plan: Arc, right_input_plan: Arc, - mapping: &HashMap, + left_mapping: &HashMap, + right_mapping: &HashMap, left_size: usize, ) -> Result> { let new_filter = nlj.filter().map(|filter| { @@ -3823,7 +3919,7 @@ fn rewrite_nested_loop_join( .iter() .map(|col_idx| match col_idx.side { JoinSide::Left => ColumnIndex { - index: mapping + index: left_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index @@ -3833,7 +3929,7 @@ fn rewrite_nested_loop_join( side: JoinSide::Left, }, JoinSide::Right => ColumnIndex { - index: mapping + index: right_mapping .iter() .find(|(old_column, _new_column)| { old_column.index() == col_idx.index + left_size @@ -4006,7 +4102,7 @@ fn rewrite_aggregate( input_plan, agg.input_schema(), ) - .map(|plan| Some(Arc::new(plan) as _)) + .map(|plan| Some(Arc::new(plan.with_limit(agg.limit())) as _)) } fn rewrite_window_aggregate( @@ -4015,7 +4111,7 @@ fn rewrite_window_aggregate( mapping: &HashMap, ) -> Result>> { let new_window = - if let Some(new_window) = update_window_exprs(&w_agg.window_expr(), mapping) { + if let Some(new_window) = update_window_exprs(w_agg.window_expr(), mapping) { new_window } else { return Ok(None); @@ -4053,7 +4149,6 @@ mod tests { use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; - use crate::execution::context::SessionContext; use crate::physical_optimizer::optimize_projections::{ update_expr_with_projection, OptimizeProjections, }; @@ -4068,11 +4163,9 @@ mod tests { use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::ExecutionPlan; - use arrow::util::pretty::print_batches; use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; - use datafusion_execution::config::SessionConfig; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{ColumnarValue, Operator}; use datafusion_physical_expr::expressions::{ @@ -4085,8 +4178,6 @@ mod tests { use datafusion_physical_plan::joins::SymmetricHashJoinExec; use datafusion_physical_plan::union::UnionExec; - use super::print_plan; - fn create_simple_csv_exec() -> Arc { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int32, true), @@ -4927,251 +5018,4 @@ mod tests { assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) } - - // #[tokio::test] - // async fn test_trivial() -> Result<()> { - // let config = SessionConfig::new() - // .with_target_partitions(2) - // .with_batch_size(4096); - // let ctx = SessionContext::with_config(config); - // let _dataframe = ctx - // .sql( - // "CREATE EXTERNAL TABLE aggregate_test_100 ( - // c1 VARCHAR NOT NULL, - // c2 TINYINT NOT NULL, - // c3 SMALLINT NOT NULL, - // c4 SMALLINT, - // c5 INT, - // c6 BIGINT NOT NULL, - // c7 SMALLINT NOT NULL, - // c8 INT NOT NULL, - // c9 BIGINT UNSIGNED NOT NULL, - // c10 VARCHAR NOT NULL, - // c11 FLOAT NOT NULL, - // c12 DOUBLE NOT NULL, - // c13 VARCHAR NOT NULL - // ) - // STORED AS CSV - // WITH HEADER ROW - // LOCATION '/Users/berkaysahin/Desktop/datafusion-upstream/datafusion/core/tests/tpch-csv/lineitem.csv'", - // ) - // .await?; - - // let dataframe = ctx - // .sql( - // "WITH indices AS ( - // SELECT 1 AS idx UNION ALL - // SELECT 2 AS idx UNION ALL - // SELECT 3 AS idx UNION ALL - // SELECT 4 AS idx UNION ALL - // SELECT 5 AS idx - // ) - // SELECT data.arr[indices.idx] as element, array_length(data.arr) as array_len, dummy - // FROM ( - // SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100 - // ) data - // CROSS JOIN indices - // ORDER BY 1", - // ) - // .await?; - // let physical_plan = dataframe.clone().create_physical_plan().await?; - // let batches = dataframe.collect().await?; - // let _ = print_plan(&physical_plan); - // let _ = print_batches(&batches); - // Ok(()) - // } - - /// TODO: Result of this query is wrong, where output schema orders is different what is should be. - /// The problem originates in `AggregateExec: mode=FinalPartitioned` at the top. - #[tokio::test] - async fn test_trivial2() -> Result<()> { - let mut config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); - let _dataframe = ctx - .sql( - "CREATE TABLE web_sales ( - ws_sold_date_sk INT NOT NULL, - ws_sold_time_sk INT NOT NULL, - ws_ship_date_sk INT NOT NULL, - ws_item_sk INT NOT NULL, - ws_bill_customer_sk INT NOT NULL, - ws_bill_cdemo_sk INT NOT NULL, - ws_bill_hdemo_sk INT NOT NULL, - ws_bill_addr_sk INT NOT NULL, - ws_ship_customer_sk INT NOT NULL, - ws_ship_cdemo_sk INT NOT NULL, - ws_ship_hdemo_sk INT NOT NULL, - ws_ship_addr_sk INT NOT NULL, - ws_web_page_sk INT NOT NULL, - ws_web_site_sk INT NOT NULL, - ws_ship_mode_sk INT NOT NULL, - ws_warehouse_sk INT NOT NULL, - ws_promo_sk INT NOT NULL, - ws_order_number BIGINT NOT NULL, - ws_quantity INT NOT NULL, - ws_wholesale_cost DECIMAL(10,2) NOT NULL, - ws_list_price DECIMAL(10,2) NOT NULL, - ws_sales_price DECIMAL(10,2) NOT NULL, - ws_ext_discount_amt DECIMAL(10,2) NOT NULL, - ws_ext_sales_price DECIMAL(10,2) NOT NULL, - ws_ext_wholesale_cost DECIMAL(10,2) NOT NULL, - ws_ext_list_price DECIMAL(10,2) NOT NULL, - ws_ext_tax DECIMAL(10,2) NOT NULL, - ws_coupon_amt DECIMAL(10,2) NOT NULL, - ws_ext_ship_cost DECIMAL(10,2) NOT NULL, - ws_net_paid DECIMAL(10,2) NOT NULL, - ws_net_paid_inc_tax DECIMAL(10,2) NOT NULL, - ws_net_paid_inc_ship DECIMAL(10,2) NOT NULL, - ws_net_paid_inc_ship_tax DECIMAL(10,2) NOT NULL, - ws_net_profit DECIMAL(10,2) NOT NULL -);", - ) - .await?; - - let _dataframe = ctx - .sql( - "CREATE TABLE catalog_sales ( - cs_sold_date_sk INT NOT NULL, - cs_sold_time_sk INT NOT NULL, - cs_ship_date_sk INT NOT NULL, - cs_bill_customer_sk INT NOT NULL, - cs_bill_cdemo_sk INT NOT NULL, - cs_bill_hdemo_sk INT NOT NULL, - cs_bill_addr_sk INT NOT NULL, - cs_ship_customer_sk INT NOT NULL, - cs_ship_cdemo_sk INT NOT NULL, - cs_ship_hdemo_sk INT NOT NULL, - cs_ship_addr_sk INT NOT NULL, - cs_call_center_sk INT NOT NULL, - cs_catalog_page_sk INT NOT NULL, - cs_ship_mode_sk INT NOT NULL, - cs_warehouse_sk INT NOT NULL, - cs_item_sk INT NOT NULL, - cs_promo_sk INT NOT NULL, - cs_order_number BIGINT NOT NULL, - cs_quantity INT NOT NULL, - cs_wholesale_cost DECIMAL(10,2) NOT NULL, - cs_list_price DECIMAL(10,2) NOT NULL, - cs_sales_price DECIMAL(10,2) NOT NULL, - cs_ext_discount_amt DECIMAL(10,2) NOT NULL, - cs_ext_sales_price DECIMAL(10,2) NOT NULL, - cs_ext_wholesale_cost DECIMAL(10,2) NOT NULL, - cs_ext_list_price DECIMAL(10,2) NOT NULL, - cs_ext_tax DECIMAL(10,2) NOT NULL, - cs_coupon_amt DECIMAL(10,2) NOT NULL, - cs_ext_ship_cost DECIMAL(10,2) NOT NULL, - cs_net_paid DECIMAL(10,2) NOT NULL, - cs_net_paid_inc_tax DECIMAL(10,2) NOT NULL, - cs_net_paid_inc_ship DECIMAL(10,2) NOT NULL, - cs_net_paid_inc_ship_tax DECIMAL(10,2) NOT NULL, - cs_net_profit DECIMAL(10,2) NOT NULL -);", - ) - .await?; - - let _dataframe = ctx - .sql( - "CREATE TABLE date_dim ( - d_date_sk INT NOT NULL, - d_date_id VARCHAR NOT NULL, - d_date DATE NOT NULL, - d_month_seq INT NOT NULL, - d_week_seq INT NOT NULL, - d_quarter_seq INT NOT NULL, - d_year INT NOT NULL, - d_dow INT NOT NULL, - d_moy INT NOT NULL, - d_dom INT NOT NULL, - d_qoy INT NOT NULL, - d_fy_year INT NOT NULL, - d_fy_quarter_seq INT NOT NULL, - d_fy_week_seq INT NOT NULL, - d_day_name VARCHAR NOT NULL, - d_quarter_name VARCHAR NOT NULL, - d_holiday VARCHAR, - d_weekend VARCHAR, - d_following_holiday VARCHAR, - d_first_dom INT NOT NULL, - d_last_dom INT NOT NULL, - d_same_day_ly INT NOT NULL, - d_same_day_lq INT NOT NULL, - d_current_day VARCHAR, - d_current_week VARCHAR, - d_current_month VARCHAR, - d_current_quarter VARCHAR, - d_current_year VARCHAR -); -", - ) - .await?; - - let dataframe = ctx - .sql( - "with wscs as - (select sold_date_sk - ,sales_price - from (select ws_sold_date_sk sold_date_sk - ,ws_ext_sales_price sales_price - from web_sales - union all - select cs_sold_date_sk sold_date_sk - ,cs_ext_sales_price sales_price - from catalog_sales)), - wswscs as - (select d_week_seq, - sum(case when (d_day_name='Sunday') then sales_price else null end) sun_sales, - sum(case when (d_day_name='Monday') then sales_price else null end) mon_sales, - sum(case when (d_day_name='Tuesday') then sales_price else null end) tue_sales, - sum(case when (d_day_name='Wednesday') then sales_price else null end) wed_sales, - sum(case when (d_day_name='Thursday') then sales_price else null end) thu_sales, - sum(case when (d_day_name='Friday') then sales_price else null end) fri_sales, - sum(case when (d_day_name='Saturday') then sales_price else null end) sat_sales - from wscs - ,date_dim - where d_date_sk = sold_date_sk - group by d_week_seq) - select d_week_seq1 - ,round(sun_sales1/sun_sales2,2) - ,round(mon_sales1/mon_sales2,2) - ,round(tue_sales1/tue_sales2,2) - ,round(wed_sales1/wed_sales2,2) - ,round(thu_sales1/thu_sales2,2) - ,round(fri_sales1/fri_sales2,2) - ,round(sat_sales1/sat_sales2,2) - from - (select wswscs.d_week_seq d_week_seq1 - ,sun_sales sun_sales1 - ,mon_sales mon_sales1 - ,tue_sales tue_sales1 - ,wed_sales wed_sales1 - ,thu_sales thu_sales1 - ,fri_sales fri_sales1 - ,sat_sales sat_sales1 - from wswscs,date_dim - where date_dim.d_week_seq = wswscs.d_week_seq and - d_year = 2001) y, - (select wswscs.d_week_seq d_week_seq2 - ,sun_sales sun_sales2 - ,mon_sales mon_sales2 - ,tue_sales tue_sales2 - ,wed_sales wed_sales2 - ,thu_sales thu_sales2 - ,fri_sales fri_sales2 - ,sat_sales sat_sales2 - from wswscs - ,date_dim - where date_dim.d_week_seq = wswscs.d_week_seq and - d_year = 2001+1) z - where d_week_seq1=d_week_seq2-53 - order by d_week_seq1;", - ) - .await?; - - let physical_plan = dataframe.clone().create_physical_plan().await?; - let batches = dataframe.collect().await?; - let _ = print_plan(&physical_plan); - let _ = print_batches(&batches); - - Ok(()) - } } diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index 9b42cca76bd0..8327720f2e50 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -156,15 +156,16 @@ impl AggregateExpr for Sum { fn with_new_expressions( self: Arc, - mut expressions: Vec>, + _expressions: Vec>, ) -> Option> { - Some(Arc::new(Self { - name: self.name.clone(), - data_type: self.data_type.clone(), - return_type: self.return_type.clone(), - expr: expressions.swap_remove(0), - nullable: self.nullable, - })) + // Some(Arc::new(Self { + // name: self.name.clone(), + // data_type: self.data_type.clone(), + // return_type: self.return_type.clone(), + // expr: expressions.swap_remove(0), + // nullable: self.nullable, + // })) + None } } diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index 58fe4728543d..d6e5ea231a1d 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -52,6 +52,20 @@ pub enum GetFieldAccessExpr { }, } +impl GetFieldAccessExpr { + pub fn exprs(&self) -> Vec> { + match self { + GetFieldAccessExpr::NamedStructField { .. } => vec![], + GetFieldAccessExpr::ListIndex { key } => vec![key.clone()], + GetFieldAccessExpr::ListRange { + start, + stop, + stride, + } => vec![start.clone(), stop.clone(), stride.clone()], + } + } +} + impl std::fmt::Display for GetFieldAccessExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { @@ -285,7 +299,9 @@ impl PhysicalExpr for GetIndexedFieldExpr { } fn children(&self) -> Vec> { - vec![self.arg.clone()] + let mut children = vec![self.arg.clone()]; + children.extend(self.field.exprs()); + children } fn with_new_children( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c116d4b96157..3baa918394ac 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -238,7 +238,7 @@ impl From for SendableRecordBatchStream { } /// Hash aggregate execution plan -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct AggregateExec { /// Aggregation mode (full, partial) mode: AggregateMode, diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 136fb39c673e..4184ce933319 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2644,14 +2644,13 @@ physical_plan GlobalLimitExec: skip=10, fetch=3 --AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] +------AggregateExec: mode=Partial, gby=[c3@1 as c3, c2@0 as c2], aggr=[], lim=[13] --------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] -------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +----------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +------------CoalescePartitionsExec +--------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------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/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 1f2870a8c92f..89b8595d4d25 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -187,18 +187,18 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum --------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] ----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] -------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +----ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t1_id@2 as t1_id] +------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)] --------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ----------CoalesceBatchesExec: target_batch_size=2 ------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +------------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,18 +221,19 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2 --------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Float64)) AS SUM(t2.t2_int * Float64(1))]] ----------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] +ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] -------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] ---------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----ProjectionExec: expr=[SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as SUM(t2.t2_int * Float64(1)) + Int64(1), t1_id@2 as t1_id] +------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] +--------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] +----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +------------CoalesceBatchesExec: target_batch_size=2 +--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -257,18 +258,19 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 -----------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] -------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) ---------------CoalesceBatchesExec: target_batch_size=2 -----------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 -------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) ---------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as SUM(t2.t2_int)] +------HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 +------------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] +--------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 +--------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) +----------------------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 group by t2_id, 'a') as t2_sum from t1 @@ -292,10 +294,10 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum ----------Aggregate: groupBy=[[t2.t2_id]], aggr=[[SUM(CAST(t2.t2_int AS Int64))]] ------------TableScan: t2 projection=[t2_id, t2_int] physical_plan -ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as t2_sum] +ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] -------ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t2_id@0 as t2_id] +----ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t1_id@2 as t1_id] +------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)] --------CoalesceBatchesExec: target_batch_size=2 ----------FilterExec: SUM(t2.t2_int)@1 < 3 ------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] @@ -303,9 +305,9 @@ ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int)@0 as t2_sum] ----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] --------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +------------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 @@ -833,6 +835,7 @@ Projection: t1.t1_int ------------Aggregate: groupBy=[[t2.t2_id, Boolean(true) AS __always_true]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] --------------TableScan: t2 projection=[t2_id] + query I rowsort select t1.t1_int from t1 where (select count(*) from t2 where t1.t1_id = t2.t2_id) < t1.t1_int ---- From b9a2dfb34949d92a146d6d194efe62aeb8e60f6a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 23 Feb 2024 13:14:08 +0300 Subject: [PATCH 13/85] Prevent projection removals causing uncertain column swaps --- .../optimize_projections.rs | 41 +++++++++++-------- .../sqllogictest/test_files/group_by.slt | 6 +-- 2 files changed, 28 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 35ac85d00b40..1d26620ae0fb 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -30,14 +30,14 @@ //! --------------- //! - Traverses the plan from root to leaves. If the node is: //! 1. Projection node, it may: -//! a) Merge it with its input projection if merge is beneficial. -//! b) Remove the projection if it is redundant. -//! c) Narrow the Projection if possible. -//! d) The projection can be nested into the source. -//! e) Do nothing, otherwise. +//! a. Merge it with its input projection if merge is beneficial. +//! b. Remove the projection if it is redundant. +//! c. Narrow the Projection if possible. +//! d. The projection can be nested into the source. +//! e. Do nothing, otherwise. //! 2. Non-Projection node: -//! a) Schema needs pruning. Insert the necessary projections to the children. -//! b) All fields are required. Do nothing. +//! a. Schema needs pruning. Insert the necessary projections to the children. +//! b. All fields are required. Do nothing. //! //! Bottom-up Phase (now resides in map_children() implementation): //! ---------------- @@ -172,10 +172,10 @@ impl ProjectionOptimizer { } /// The function tries 4 cases: - /// 1) If the input plan is also a projection, they can be merged into one projection. - /// 2) The projection can be removed. - /// 3) The projection can get narrower. - /// 4) The projection can be embedded into the source. + /// 1. If the input plan is also a projection, they can be merged into one projection. + /// 2. The projection can be removed. + /// 3. The projection can get narrower. + /// 4. The projection can be embedded into the source. /// If none of them is possible, it remains unchanged. pub fn optimize_projections(mut self) -> Result { let projection_input = self.plan.children(); @@ -278,8 +278,8 @@ impl ProjectionOptimizer { /// 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. + /// 1. Projection must have all column expressions without aliases. + /// 2. Projection input is fully required by the projection output requirements. fn try_remove_projection(mut self) -> Transformed { // It must be a projection let projection_exec = @@ -306,9 +306,14 @@ impl ProjectionOptimizer { // If all fields of the input are necessary, we can remove the projection. let input_columns = collect_columns_in_plan_schema(projection_exec.input()); + let input_col_names = input_columns + .iter() + .map(|col| col.name().to_string()) + .collect::>(); if input_columns .iter() .all(|input_column| projection_requires.contains(input_column)) + && input_col_names.len() == input_columns.len() { let new_mapping = self .required_columns @@ -2983,9 +2988,14 @@ impl ProjectionOptimizer { expr.as_any().downcast_ref::().unwrap().clone() }) .collect::>(); + 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; @@ -3145,7 +3155,6 @@ impl PhysicalOptimizerRule for OptimizeProjections { plan: Arc, _config: &ConfigOptions, ) -> Result> { - // print_plan(&plan); // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); let mut optimizer = ProjectionOptimizer::new_default(plan); @@ -3444,14 +3453,14 @@ enum RewriteState { /// The function operates in two modes: /// -/// 1) When `sync_with_child` is `true`: +/// 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`: +/// 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 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 54da1dd22355..5cc5a3fae544 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2875,9 +2875,9 @@ physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] --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)@5 as last_rate] ----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)] -------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] ---------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount] -----------CoalesceBatchesExec: target_batch_size=8192 +------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] +--------CoalesceBatchesExec: target_batch_size=8192 +----------ProjectionExec: expr=[sn@0 as sn, amount@3 as amount, zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency] ------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 --------------MemoryExec: partitions=1, partition_sizes=[1] --------------MemoryExec: partitions=1, partition_sizes=[1] From f75cf6b9aced024ccd6758e42e515785caaa0b85 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 26 Feb 2024 16:03:58 +0300 Subject: [PATCH 14/85] Minor changes --- .../optimize_projections.rs | 388 +++++++++--------- 1 file changed, 198 insertions(+), 190 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 1d26620ae0fb..9eacbf0582bb 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -47,7 +47,7 @@ //! the leaves to the root, updating the indices of columns in the plans by //! referencing these mapping records. After the top-down phase, also some //! unnecessary projections may emerge. When projections check its input schema -//! mapping, it can remove itself and assign new schema mapping to the new node +//! mapping, it can remove itself and assign new schema mapping to the new node, //! which was the projection's input formerly. use std::collections::{HashMap, HashSet}; @@ -65,7 +65,7 @@ use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::DataFusionError; -use datafusion_common::{internal_err, JoinSide, JoinType}; +use datafusion_common::{JoinSide, JoinType}; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::window::WindowExpr; @@ -100,7 +100,6 @@ use itertools::Itertools; pub struct ProjectionOptimizer { pub plan: Arc, /// The node above expects it can reach these columns. - /// Note: This set can be built on column indices rather than column expressions. pub required_columns: HashSet, /// The nodes above will be updated according to these mathces. First element indicates /// the initial column index, and the second element is for the updated version. @@ -113,7 +112,7 @@ pub struct ProjectionOptimizer { type ColumnRequirements = HashMap; impl ProjectionOptimizer { - /// Constructs the empty graph according to the plan. All state information is empty initially. + /// 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 { @@ -127,48 +126,23 @@ impl ProjectionOptimizer { /// 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(self) -> Result { - // print_plan(&self.plan); - // println!("self reqs: {:?}", self.required_columns); - // println!("self map: {:?}", self.schema_mapping); - // self.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - + fn adjust_node_with_requirements(mut self) -> Result { // If the node is a source provdider, no need a change. if self.children_nodes.is_empty() { + // We also clean the requirements, since we would like + // to left a payload-free nodes after the rule finishes. + self.required_columns.clear(); return Ok(self); } - let x = if self.plan.as_any().is::() { + if self.plan.as_any().is::() { // If the node is a projection, it is analyzed and may be rewritten - // in a most effective way, or even removed. + // to make the projection more efficient, or even it may be removed. self.optimize_projections() } else { - // If the node corresponds to any other plan, a projection may be inserted to its input. + // If the node is any other plan, a projection may be inserted to its input. self.try_projection_insertion() - }?; - - // print_plan(&x.plan); - // println!("self reqs: {:?}", x.required_columns); - // println!("self map: {:?}", x.schema_mapping); - // x.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // x.children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // x.children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - - Ok(x) + } } /// The function tries 4 cases: @@ -177,7 +151,7 @@ impl ProjectionOptimizer { /// 3. The projection can get narrower. /// 4. The projection can be embedded into the source. /// If none of them is possible, it remains unchanged. - pub fn optimize_projections(mut self) -> Result { + fn optimize_projections(mut self) -> Result { let projection_input = self.plan.children(); let projection_input = projection_input[0].as_any(); @@ -222,15 +196,8 @@ impl ProjectionOptimizer { // If none of them possible, we will continue to next node. Output requirements // of the projection in terms of projection input are inserted to child node. - let Some(projection_plan) = self.plan.as_any().downcast_ref::() - else { - return internal_err!( - "\"optimize_projections\" subrule must be used on ProjectionExec's." - ); - }; - - // If there is nothing that could be better, insert the child requirements and continue. - + let projection_plan = + self.plan.as_any().downcast_ref::().unwrap(); self.children_nodes[0].required_columns = self .required_columns .iter() @@ -239,8 +206,8 @@ impl ProjectionOptimizer { Ok(self) } - /// Unifies `projection` with its input, which is also a [`ProjectionExec`], if it is beneficial. - fn try_unifying_projections(mut self) -> Result> { + /// If it is beneficial. unifies the projection and its input, which is also a [`ProjectionExec`]. + fn try_unifying_projections(mut self) -> Result> { // These are known to be a ProjectionExec. let projection = self.plan.as_any().downcast_ref::().unwrap(); let child_projection = self.children_nodes[0] @@ -266,7 +233,7 @@ impl ProjectionOptimizer { let new_plan = ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) .map(|e| Arc::new(e) as _)?; - Ok(Transformed::Yes(ProjectionOptimizer { + Ok(Transformed::Yes(Self { plan: new_plan, // Schema of the projection does not change, // so no need any update on state variables. @@ -280,60 +247,36 @@ impl ProjectionOptimizer { /// the projection can be safely removed: /// 1. Projection must have all column expressions without aliases. /// 2. Projection input is fully required by the projection output requirements. - fn try_remove_projection(mut self) -> Transformed { + fn try_remove_projection(mut self) -> Transformed { // It must be a projection let projection_exec = self.plan.as_any().downcast_ref::().unwrap(); - // The projection must have all column expressions without aliases. if !all_alias_free_columns(projection_exec.expr()) { return Transformed::No(self); } + // The expressions are known to be all columns. - let projection_columns = projection_exec - .expr() - .iter() - .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) - .cloned() - .collect::>(); + let projection_columns = + downcast_projected_exprs_to_columns(projection_exec).unwrap(); // Input requirements of the projection in terms of projection's parent requirements: - let projection_requires = self - .required_columns - .iter() - .map(|column| projection_columns[column.index()].clone()) - .collect::>(); + let projection_requires = + map_parent_reqs_to_input_reqs(&self.required_columns, &projection_columns); // If all fields of the input are necessary, we can remove the projection. let input_columns = collect_columns_in_plan_schema(projection_exec.input()); - let input_col_names = input_columns - .iter() - .map(|col| col.name().to_string()) - .collect::>(); - if input_columns - .iter() - .all(|input_column| projection_requires.contains(input_column)) - && input_col_names.len() == input_columns.len() - { - let new_mapping = self - .required_columns - .into_iter() - .filter_map(|column| { - let col_ind = column.index(); - if column != projection_columns[col_ind] { - Some((column, projection_columns[col_ind].clone())) - } else { - None - } - }) - .collect(); - - let replaced_child = self.children_nodes.swap_remove(0); + 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); Transformed::Yes(ProjectionOptimizer { - plan: replaced_child.plan, + plan: new_current_node.plan, required_columns: projection_requires, - schema_mapping: new_mapping, - children_nodes: replaced_child.children_nodes, + schema_mapping, + children_nodes: new_current_node.children_nodes, }) } else { Transformed::No(self) @@ -347,54 +290,45 @@ impl ProjectionOptimizer { let projection_exec = self.plan.as_any().downcast_ref::().unwrap(); - // Check for the projection output if it has any redundant elements. - let projection_output_columns = projection_exec - .expr() - .iter() - .enumerate() - .map(|(i, (_e, a))| Column::new(a, i)) - .collect::>(); - let used_indices = projection_output_columns - .iter() - .filter(|&p_out| self.required_columns.contains(p_out)) - .map(|p_out| p_out.index()) - .collect::>(); - - if used_indices.len() == projection_output_columns.len() { + let requirement_map = self.analyze_requirements(); + let (used_columns, unused_columns) = split_column_requirements(&requirement_map); + if unused_columns.is_empty() { // All projected items are used. return Ok(Transformed::No(self)); } - - // New projected expressions are rewritten according to used indices. - let new_projection = used_indices + let projected_exprs = projection_exec + .expr() .iter() - .map(|i| projection_exec.expr()[*i].clone()) + .enumerate() + .filter_map(|(idx, (expr, alias))| { + if used_columns + .iter() + .map(|column| column.index()) + .collect::>() + .contains(&idx) + { + Some((expr.clone(), alias.clone())) + } else { + None + } + }) .collect::>(); - - // Construct the mapping. - let mut schema_mapping = HashMap::new(); - for (new_idx, old_idx) in used_indices.iter().enumerate() { - if new_idx != *old_idx { - schema_mapping.insert( - projection_output_columns[*old_idx].clone(), - projection_output_columns[new_idx].clone(), - ); - } - } + let new_mapping = + calculate_column_mapping(&self.required_columns, &unused_columns); let new_projection_plan = Arc::new(ProjectionExec::try_new( - new_projection.clone(), + projected_exprs.clone(), self.children_nodes[0].plan.clone(), )?); let new_projection_requires = self .required_columns .iter() - .map(|col| schema_mapping.get(col).cloned().unwrap_or(col.clone())) + .map(|col| new_mapping.get(col).cloned().unwrap_or(col.clone())) .collect(); let mut new_node = ProjectionOptimizer { plan: new_projection_plan, required_columns: new_projection_requires, - schema_mapping, + schema_mapping: new_mapping, children_nodes: self.children_nodes, }; @@ -402,7 +336,7 @@ impl ProjectionOptimizer { new_node.children_nodes[0].required_columns = self .required_columns .iter() - .flat_map(|column| collect_columns(&new_projection[column.index()].0)) + .flat_map(|column| collect_columns(&projected_exprs[column.index()].0)) .collect::>(); Ok(Transformed::Yes(new_node)) @@ -2357,7 +2291,7 @@ impl ProjectionOptimizer { ) -> Result<(Self, HashMap)> { // During the iteration, we construct the ProjectionExec with required columns as the new child, // and also collect the unused columns to store the index changes after removal of some columns. - let (used_columns, unused_columns) = split_column_reqs(&requirement_map); + let (used_columns, unused_columns) = split_column_requirements(&requirement_map); let mut projected_exprs = convert_projection_exprs(used_columns); projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() @@ -2388,7 +2322,7 @@ impl ProjectionOptimizer { ) -> Result<(Vec, HashMap)> { // During the iteration, we construct the ProjectionExec's with required columns as the new children, // and also collect the unused columns to store the index changes after removal of some columns. - let (used_columns, unused_columns) = split_column_reqs(&requirement_map); + let (used_columns, unused_columns) = split_column_requirements(&requirement_map); let mut projected_exprs = convert_projection_exprs(used_columns); projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() @@ -2436,7 +2370,8 @@ impl ProjectionOptimizer { ) -> Result<(Self, HashMap)> { // During the iteration, we construct the ProjectionExec with required columns as the new child, // and also collect the unused columns to store the index changes after removal of some columns. - let (used_columns, unused_columns) = split_column_reqs(&requirement_map_left); + let (used_columns, unused_columns) = + split_column_requirements(&requirement_map_left); let mut projected_exprs = convert_projection_exprs(used_columns); projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() @@ -2526,7 +2461,7 @@ impl ProjectionOptimizer { let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); - let (used_columns, mut unused_columns) = split_column_reqs(&base); + let (used_columns, mut unused_columns) = split_column_requirements(&base); let projected_exprs = convert_projection_exprs(used_columns); window.iter().for_each(|(col, used)| { @@ -2564,7 +2499,7 @@ impl ProjectionOptimizer { .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); // let mut unused_columns = HashSet::new(); - let (required_cols, mut unused_columns) = split_column_reqs(&base); + let (required_cols, mut unused_columns) = split_column_requirements(&base); let projected_exprs = convert_projection_exprs(required_cols); window.iter().for_each(|(col, used)| { if !used { @@ -3238,7 +3173,9 @@ fn satisfy_initial_schema( po: ProjectionOptimizer, initial_requirements: HashSet, ) -> Result { - if collect_columns_in_plan_schema(&po.plan) == initial_requirements { + if collect_columns_in_plan_schema(&po.plan) == initial_requirements + && po.schema_mapping.is_empty() + { // The initial schema is already satisfied, no further action required. Ok(po) } else { @@ -3412,10 +3349,12 @@ fn calculate_column_mapping( } /// Given a `ColumnRequirements`, it separates the required and redundant columns. -fn split_column_reqs(reqs: &ColumnRequirements) -> (HashSet, HashSet) { +fn split_column_requirements( + requirements: &ColumnRequirements, +) -> (HashSet, HashSet) { let mut required = HashSet::new(); let mut unused = HashSet::new(); - for (col, is_req) in reqs { + for (col, is_req) in requirements { if *is_req { required.insert(col.clone()); } else { @@ -3428,13 +3367,13 @@ fn split_column_reqs(reqs: &ColumnRequirements) -> (HashSet, HashSet, + columns: HashSet, ) -> Vec<(Arc, String)> { - let result = cols + let result = columns .into_iter() - .map(|col| { - let name = col.name().to_string(); - (Arc::new(col) as Arc, name) + .map(|column| { + let name = column.name().to_string(); + (Arc::new(column) as Arc, name) }) .collect::>(); result @@ -3630,61 +3569,6 @@ fn update_column_index( new_expr } -/// Collects all fields of the schema for a given plan in [`Column`] form. -fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { - plan.schema() - .fields() - .iter() - .enumerate() - .map(|(i, f)| Column::new(f.name(), i)) - .collect() -} - -/// Collects all columns in the join's equivalence and non-equivalence conditions as they are seen at the join output. -/// This means that columns from left table appear as they are, and right table column indices increased by left table size. -fn collect_columns_in_join_conditions( - on: &[(PhysicalExprRef, PhysicalExprRef)], - filter: Option<&JoinFilter>, - left_size: usize, - join_left_schema: SchemaRef, - join_right_schema: SchemaRef, -) -> HashSet { - let equivalence_columns = on - .iter() - .flat_map(|(col_left, col_right)| { - let left_columns = collect_columns(col_left); - let right_columns = collect_columns(col_right); - let 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() -} - /// Updates the equivalence conditions of the joins according to the new indices of columns. fn update_equivalence_conditions( on: &[(PhysicalExprRef, PhysicalExprRef)], @@ -3771,6 +3655,61 @@ fn update_non_equivalence_conditions( }) } +/// Collects all fields of the schema for a given plan in [`Column`] form. +fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { + plan.schema() + .fields() + .iter() + .enumerate() + .map(|(i, f)| Column::new(f.name(), i)) + .collect() +} + +/// Collects all columns in the join's equivalence and non-equivalence conditions as they are seen at the join output. +/// This means that columns from left table appear as they are, and right table column indices increased by left table size. +fn collect_columns_in_join_conditions( + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, + left_size: usize, + join_left_schema: SchemaRef, + join_right_schema: SchemaRef, +) -> HashSet { + let equivalence_columns = on + .iter() + .flat_map(|(col_left, col_right)| { + let left_columns = collect_columns(col_left); + let right_columns = collect_columns(col_right); + let 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() +} + /// Calculates how many index of the given column decreases becasue of /// the removed columns which reside on the left side of that given column. fn removed_column_count( @@ -3796,6 +3735,75 @@ fn removed_column_count( left_skipped_columns } +/// Downcasts all expression's PhysicalExpr part in a projection. Returns error if any downcast fails. +fn downcast_projected_exprs_to_columns( + projection: &ProjectionExec, +) -> Result> { + let Some(columns) = projection + .expr() + .iter() + .map(|(expr, _)| expr.as_any().downcast_ref::()) + .collect::>>() + else { + return Err(DataFusionError::Internal("PhysicalExpr which is tried to be downcasted is not a Column. + `downcast_projected_exprs_to_columns` can be used after the check of `all_alias_free_columns`".to_string())); + }; + + Ok(columns.into_iter().cloned().collect()) +} + +/// Maps the indices of columns which are required from a projection node +/// such that the projection inserts the same requirements into 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: &HashSet, + projection_columns: &[Column], +) -> HashSet { + requirements + .iter() + .map(|column| projection_columns[column.index()].clone()) + .collect::>() +} + +/// Compares the fields of a projection input schema with projection requirements. +/// If all input fields are required, then the function returns `true`. +fn all_input_columns_required( + input_columns: &HashSet, + projection_requires: &HashSet, +) -> bool { + input_columns + .iter() + .all(|input_column| projection_requires.contains(input_column)) +} + +/// When we remove a redundant projection from a plan, parent operators are informed +/// about the index changes caused by the removal of this projection. This function +/// iterates the columns and records the changes for all of them after the removal. +/// If a change is observed, the old and new index values are inserted into a hashmap. +fn index_changes_after_projection_removal( + columns: HashSet, + projection_columns: &[Column], +) -> HashMap { + 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() +} + fn rewrite_projection( projection: &ProjectionExec, input_plan: Arc, From 8b06852e9e6a5565bf711be1e74ad5651afd5d66 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 26 Feb 2024 16:27:35 +0300 Subject: [PATCH 15/85] Minor changes --- datafusion/physical-expr/src/aggregate/sum.rs | 14 -------------- .../test_files/agg_func_substitute.slt | 4 ++-- 2 files changed, 2 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/sum.rs b/datafusion/physical-expr/src/aggregate/sum.rs index 8327720f2e50..6cf2810ce588 100644 --- a/datafusion/physical-expr/src/aggregate/sum.rs +++ b/datafusion/physical-expr/src/aggregate/sum.rs @@ -153,20 +153,6 @@ impl AggregateExpr for Sum { } downcast_sum!(self, helper) } - - fn with_new_expressions( - self: Arc, - _expressions: Vec>, - ) -> Option> { - // Some(Arc::new(Self { - // name: self.name.clone(), - // data_type: self.data_type.clone(), - // return_type: self.return_type.clone(), - // expr: expressions.swap_remove(0), - // nullable: self.nullable, - // })) - None - } } impl PartialEq for Sum { diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index ff485e97ca0c..61148980f2ff 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -43,7 +43,7 @@ Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1 --Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] ----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1))@1 as result] --AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=8192 @@ -63,7 +63,7 @@ Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1 --Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] ----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1))@1 as result] --AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=8192 From e632f7aa268e5eeccc96480cba1b0fa805eac180 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 27 Feb 2024 10:48:13 +0300 Subject: [PATCH 16/85] Update optimize_projections.rs --- .../optimize_projections.rs | 87 +------------------ 1 file changed, 2 insertions(+), 85 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9eacbf0582bb..9db66b915301 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -2994,19 +2994,6 @@ impl TreeNode for ProjectionOptimizer { where F: FnMut(Self) -> Result, { - // print_plan(&self.plan); - // println!("self reqs: {:?}", self.required_columns); - // println!("self map: {:?}", self.schema_mapping); - // self.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - if self.children_nodes.is_empty() { Ok(self) } else { @@ -3016,19 +3003,6 @@ impl TreeNode for ProjectionOptimizer { .map(transform) .collect::>>()?; - // print_plan(&self.plan); - // println!("self reqs: {:?}", self.required_columns); - // println!("self map: {:?}", self.schema_mapping); - // self.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - self = match self.index_updater()? { Transformed::Yes(updated) => updated, Transformed::No(not_rewritable) => { @@ -3036,39 +3010,13 @@ impl TreeNode for ProjectionOptimizer { } }; - // print_plan(&self.plan); - // println!("self reqs: {:?}", self.required_columns); - // println!("self map: {:?}", self.schema_mapping); - // self.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - // 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()?; - - // print_plan(&self.plan); - // println!("self reqs: {:?}", self.required_columns); - // println!("self map: {:?}", self.schema_mapping); - // self.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // self.children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - + Ok(self) } } @@ -3098,40 +3046,9 @@ impl PhysicalOptimizerRule for OptimizeProjections { let mut optimized = optimizer.transform_down(&|o| { o.adjust_node_with_requirements().map(Transformed::Yes) })?; - // print_plan(&optimized.plan); - // println!("self reqs: {:?}", optimized.required_columns); - // println!("self map: {:?}", optimized.schema_mapping); - // optimized - // .children_nodes - // .iter() - // .for_each(|c: &ProjectionOptimizer| { - // print_plan(&c.plan); - // }); - // optimized - // .children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // optimized - // .children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); - // Ensure the final optimized plan satisfies the initial schema requirements. + // Ensure the final optimized plan satisfies the initial schema requirements. optimized = satisfy_initial_schema(optimized, initial_requirements)?; - // print_plan(&optimized.plan); - // println!("self reqs: {:?}", optimized.required_columns); - // println!("self map: {:?}", optimized.schema_mapping); - // optimized.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // }); - // optimized - // .children_nodes - // .iter() - // .for_each(|c| println!("child reqs: {:?}", c.required_columns)); - // optimized - // .children_nodes - // .iter() - // .for_each(|c| println!("child map: {:?}", c.schema_mapping)); // TODO: Remove this check to tests crosscheck_helper(optimized.clone())?; From 3bda47413fe70687f6dd71510c0a8567efe37ede Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Tue, 27 Feb 2024 11:21:15 +0300 Subject: [PATCH 17/85] Mini Review before unwraps - Part 1 --- .../core/src/physical_optimizer/optimize_projections.rs | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9db66b915301..34d0c7a954a3 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -98,12 +98,14 @@ use itertools::Itertools; /// 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: HashSet, - /// The nodes above will be updated according to these mathces. First element indicates + /// 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: HashMap, + /// Children nodes pub children_nodes: Vec, } @@ -127,10 +129,10 @@ impl ProjectionOptimizer { /// 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 provdider, no need a change. + // 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 left a payload-free nodes after the rule finishes. + // to leave payload-free nodes after the rule finishes. self.required_columns.clear(); return Ok(self); } From 4b4054b5dc198684fafa3029f226831f55aba604 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Tue, 27 Feb 2024 11:46:11 +0300 Subject: [PATCH 18/85] Continue on review - Part 2 --- .../optimize_projections.rs | 265 ++++++++++++++---- .../src/joins/sort_merge_join.rs | 8 +- 2 files changed, 212 insertions(+), 61 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 34d0c7a954a3..9579c1169334 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -498,6 +498,17 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `CoalesceBatchesExec` node. + /// + /// This method checks if there is any redundancy in the execution plan based on the requirements + /// of the `CoalesceBatchesExec` node. If all columns are required, it updates the required columns of the child node. + /// Otherwise, it inserts a new projection to optimize the plan. + /// + /// # Arguments + /// * `coal_batches`: Reference to a `CoalesceBatchesExec` object. + /// + /// # Returns + /// * `Result`: The modified `ProjectionOptimizer` after potentially inserting a projection. fn try_insert_below_coalesce_batches( mut self, coal_batches: &CoalesceBatchesExec, @@ -525,6 +536,13 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `CoalescePartitionsExec` node. + /// + /// Similar to `try_insert_below_coalesce_batches`, this method analyzes the requirements of the + /// `CoalescePartitionsExec` node and modifies the execution plan accordingly. + /// + /// # Returns + /// * `Result`: The potentially modified `ProjectionOptimizer`. fn try_insert_below_coalesce_partitions(mut self) -> Result { // CoalescePartitionsExec does not change requirements. We can directly check whether there is a redundancy. let requirement_map = self.analyze_requirements(); @@ -545,9 +563,19 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `GlobalLimitExec` node. + /// + /// Analyzes the requirements imposed by `GlobalLimitExec` and optimizes the plan by potentially + /// inserting a projection node to reduce the number of columns processed. + /// + /// # Arguments + /// * `global_limit`: Reference to a `GlobalLimitExec` object. + /// + /// # Returns + /// * `Result`: The modified `ProjectionOptimizer`. fn try_insert_below_global_limit( mut self, - glimit: &GlobalLimitExec, + global_limit: &GlobalLimitExec, ) -> Result { // GlobalLimitExec does not change requirements. We can directly check whether there is a redundancy. let requirement_map = self.analyze_requirements(); @@ -559,8 +587,8 @@ impl ProjectionOptimizer { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; let plan = Arc::new(GlobalLimitExec::new( new_child.plan.clone(), - glimit.skip(), - glimit.fetch(), + global_limit.skip(), + global_limit.fetch(), )) as _; self = ProjectionOptimizer { @@ -573,9 +601,18 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `LocalLimitExec` node. + /// + /// Optimizes the plan considering the requirements of `LocalLimitExec`, potentially inserting a projection node. + /// + /// # Arguments + /// * `limit`: Reference to a `LocalLimitExec` object. + /// + /// # Returns + /// * `Result`: The updated `ProjectionOptimizer`. fn try_insert_below_local_limit( mut self, - llimit: &LocalLimitExec, + limit: &LocalLimitExec, ) -> Result { // LocalLimitExec does not change requirements. We can directly check whether there is a redundancy. let requirement_map = self.analyze_requirements(); @@ -585,8 +622,7 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; let plan = - Arc::new(LocalLimitExec::new(new_child.plan.clone(), llimit.fetch())) - as _; + Arc::new(LocalLimitExec::new(new_child.plan.clone(), limit.fetch())) as _; self = ProjectionOptimizer { plan, @@ -598,6 +634,16 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `FilterExec` node. + /// + /// Extends the required columns with those in the filter's predicate and optimizes the plan, potentially inserting + /// a projection node. + /// + /// # Arguments + /// * `filter`: Reference to a `FilterExec` object. + /// + /// # Returns + /// * `Result`: The optimized `ProjectionOptimizer`. fn try_insert_below_filter( mut self, filter: &FilterExec, @@ -628,6 +674,16 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `RepartitionExec` node. + /// + /// If `RepartitionExec` involves a hash repartition, it extends the requirements with the columns in the hashed expressions. + /// The method then optimizes the execution plan accordingly. + /// + /// # Arguments + /// * `repartition`: Reference to a `RepartitionExec` object. + /// + /// # Returns + /// * `Result`: The potentially updated `ProjectionOptimizer`. fn try_insert_below_repartition( mut self, repartition: &RepartitionExec, @@ -665,6 +721,16 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `SortExec` node. + /// + /// Extends the requirements with columns involved in the sort expressions and optimizes the execution plan, + /// potentially inserting a projection node. + /// + /// # Arguments + /// * `sort`: Reference to a `SortExec` object. + /// + /// # Returns + /// * `Result`: The optimized `ProjectionOptimizer`. fn try_insert_below_sort(mut self, sort: &SortExec) -> Result { // SortExec extends the requirements with the columns in its sort expressions. self.required_columns.extend( @@ -697,6 +763,16 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `SortPreservingMergeExec` node. + /// + /// Similar to `try_insert_below_sort`, it extends the requirements with columns in the sort expressions and + /// optimizes the plan accordingly. + /// + /// # Arguments + /// * `sortp_merge`: Reference to a `SortPreservingMergeExec` object. + /// + /// # Returns + /// * `Result`: The modified `ProjectionOptimizer`. fn try_insert_below_sort_preserving_merge( mut self, sortp_merge: &SortPreservingMergeExec, @@ -733,6 +809,16 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below a `UnionExec` node in the query execution plan. + /// + /// This method checks the requirements of the current execution plan to determine if there is any redundancy + /// when it comes to column usage in the context of a `UnionExec`. If all columns are required as per the + /// requirement map, the method updates the required columns for all child nodes accordingly. If not all + /// columns are required, it inserts new projection nodes to optimize the plan, leading to a more efficient + /// execution by reducing unnecessary data processing. + /// + /// # Returns + /// * `Result`: The optimized `ProjectionOptimizer` after potentially inserting projection nodes. fn try_insert_below_union(mut self) -> Result { // UnionExec does not change requirements. We can directly check whether there is a redundancy. let requirement_map = self.analyze_requirements(); @@ -758,6 +844,15 @@ impl ProjectionOptimizer { Ok(self) } + /// Attempts to insert a projection node below an `InterleaveExec` node in the query execution plan. + /// + /// Similar to `try_insert_below_union`, this method analyzes the requirements of the `InterleaveExec` node and + /// modifies the execution plan accordingly. If all columns are required, it updates the required columns for + /// each child node. Otherwise, it inserts new projection nodes for optimization. This process can lead to a + /// more efficient execution by minimizing the data processed in the context of interleaved execution. + /// + /// # Returns + /// * `Result`: The potentially modified `ProjectionOptimizer` after the optimization process. fn try_insert_below_interleave(mut self) -> Result { let requirement_map = self.analyze_requirements(); if all_columns_required(&requirement_map) { @@ -782,14 +877,31 @@ impl ProjectionOptimizer { 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. + /// + /// # Arguments + /// * `cj`: Reference to the `CrossJoinExec` node in the plan. + /// + /// # Returns + /// * `Result`: The updated `ProjectionOptimizer` after potentially inserting projection nodes. fn try_insert_below_cross_join( mut self, cj: &CrossJoinExec, ) -> Result { let left_size = cj.left().schema().fields().len(); // CrossJoinExec does not add new requirements. - let (analyzed_join_left, analyzed_join_right) = - self.analyze_requirements_of_joins(left_size); + 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), @@ -882,6 +994,27 @@ impl ProjectionOptimizer { 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. + /// + /// # Arguments + /// + /// * `hj`: Reference to a HashJoinExec node representing the join operation in the query plan. + /// + /// # Returns + /// + /// Returns `Result`, which is an updated ProjectionOptimizer + /// instance after potentially adding projections below the HashJoinExec node. + /// On success, it contains the modified ProjectionOptimizer with new projections + /// and updated plan. On failure, it returns an error indicating the issue encountered + /// during the operation. fn try_insert_below_hash_join( mut self, hj: &HashJoinExec, @@ -906,8 +1039,12 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.schema(), self.children_nodes[1].plan.schema(), )); - let (analyzed_join_left, analyzed_join_right) = - self.analyze_requirements_of_joins(left_size); + let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( + hj.left(), + hj.right(), + &self.required_columns, + left_size, + ); match hj.join_type() { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { @@ -1196,8 +1333,12 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.schema(), self.children_nodes[1].plan.schema(), )); - let (analyzed_join_left, analyzed_join_right) = - self.analyze_requirements_of_joins(left_size); + 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 => { @@ -1440,8 +1581,12 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.schema(), self.children_nodes[1].plan.schema(), )); - let (analyzed_join_left, analyzed_join_right) = - self.analyze_requirements_of_joins(left_size); + 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 => { @@ -1687,8 +1832,12 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.schema(), self.children_nodes[1].plan.schema(), )); - let (analyzed_join_left, analyzed_join_right) = - self.analyze_requirements_of_joins(left_size); + 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 => { @@ -2243,46 +2392,6 @@ impl ProjectionOptimizer { requirement_map } - /// Compares the columns required from the left/right child and existing columns in the left/right - /// child. If there is any redundant field, it returns the mapping of columns whether it is required - /// or not. If there is no redundancy, it returns `None` for that child. Caller side must ensure - /// that the join node extends its own requirements if the node's plan can introduce new requirements. - /// Each column refers to its own table schema index, not to the join output schema. - fn analyze_requirements_of_joins( - &self, - left_size: usize, - ) -> (ColumnRequirements, ColumnRequirements) { - let columns_in_schema = - collect_columns_in_plan_schema(&self.children_nodes[0].plan) - .into_iter() - .chain( - collect_columns_in_plan_schema(&self.children_nodes[1].plan) - .into_iter() - .map(|col| Column::new(col.name(), col.index() + left_size)), - ); - let requirement_map = columns_in_schema - .into_iter() - .map(|col| { - if self.required_columns.contains(&col) { - (col, true) - } else { - (col, false) - } - }) - .collect::>(); - - let (requirement_map_left, mut requirement_map_right) = requirement_map - .into_iter() - .partition::, _>(|(col, _)| col.index() < left_size); - - requirement_map_right = requirement_map_right - .into_iter() - .map(|(col, used)| (Column::new(col.name(), col.index() - left_size), used)) - .collect::>(); - - (requirement_map_left, requirement_map_right) - } - /// If a node is known to have redundant columns, we need to insert a projection to its input. /// This function takes this node and requirement mapping of this node. Then, defines the projection /// and constructs the new subtree. The returned objects are the new tree starting from the inserted @@ -3018,12 +3127,54 @@ impl TreeNode for ProjectionOptimizer { // 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) } } } +/// Compares the columns required from the left/right child and existing columns in the left/right +/// child. If there is any redundant field, it returns the mapping of columns whether it is required +/// or not. If there is no redundancy, it returns `None` for that child. Caller side must ensure +/// that the join node extends its own requirements if the node's plan can introduce new requirements. +/// Each column refers to its own table schema index, not to the join output schema. +fn analyze_requirements_of_joins( + left_child: &Arc, + right_child: &Arc, + required_columns: &HashSet, + 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) +} + #[derive(Default)] pub struct OptimizeProjections {} diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 7af614e53491..9b99360d1117 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -200,16 +200,16 @@ impl SortMergeJoinExec { &self.on } - pub fn right(&self) -> &dyn ExecutionPlan { - self.right.as_ref() + pub fn right(&self) -> &Arc { + &self.right } pub fn join_type(&self) -> JoinType { self.join_type } - pub fn left(&self) -> &dyn ExecutionPlan { - self.left.as_ref() + pub fn left(&self) -> &Arc { + &self.left } } From 5fb5202a017a26706ceddcce9253b99b3a4f31e9 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Tue, 27 Feb 2024 13:42:45 +0300 Subject: [PATCH 19/85] Adding comments - Review Part 3 --- .../optimize_projections.rs | 251 ++++++++++++++++-- 1 file changed, 226 insertions(+), 25 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9579c1169334..4093a990aaa1 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -2487,10 +2487,8 @@ impl ProjectionOptimizer { projected_exprs.sort_by_key(|(expr, _alias)| { expr.as_any().downcast_ref::().unwrap().index() }); - let inserted_projection = Arc::new(ProjectionExec::try_new( - projected_exprs.clone(), - self.plan.children()[children_index].clone(), - )?) as _; + + let child_plan = self.plan.children().remove(children_index); let required_columns = projected_exprs .iter() @@ -2499,6 +2497,9 @@ impl ProjectionOptimizer { let new_mapping = calculate_column_mapping(&required_columns, &unused_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 { @@ -3133,11 +3134,26 @@ impl TreeNode for ProjectionOptimizer { } } -/// Compares the columns required from the left/right child and existing columns in the left/right -/// child. If there is any redundant field, it returns the mapping of columns whether it is required -/// or not. If there is no redundancy, it returns `None` for that child. Caller side must ensure -/// that the join node extends its own requirements if the node's plan can introduce new requirements. -/// Each column refers to its own table schema index, not to the join output schema. +/// 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. +/// Each column in the requirement maps corresponds to its own table schema index, not to the join output schema. +/// +/// # 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, @@ -3725,7 +3741,13 @@ fn update_non_equivalence_conditions( }) } -/// Collects all fields of the schema for a given plan in [`Column`] form. +/// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. +/// +/// # Arguments +/// * `plan`: Reference to an Arc of a dynamic ExecutionPlan trait object. +/// +/// # Returns +/// A `HashSet` containing all columns from the plan's schema. fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { plan.schema() .fields() @@ -3735,8 +3757,18 @@ fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet` containing all columns from the join conditions. fn collect_columns_in_join_conditions( on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, @@ -3780,8 +3812,14 @@ fn collect_columns_in_join_conditions( .collect() } -/// Calculates how many index of the given column decreases becasue of -/// the removed columns which reside on the left side of that given column. +/// Calculates the count of removed (unused) columns that precede a given column index. +/// +/// # Arguments +/// * `requirement_map`: Reference to a ColumnRequirements map. +/// * `column_index`: The index of the column in question. +/// +/// # Returns +/// The number of removed columns before the given column index. fn removed_column_count( requirement_map: &ColumnRequirements, column_index: usize, @@ -3805,7 +3843,14 @@ fn removed_column_count( left_skipped_columns } -/// Downcasts all expression's PhysicalExpr part in a projection. Returns error if any downcast fails. +/// Downcast all expressions in a projection to their [`Column`] parts, returning an error if any downcast fails. +/// +/// # Arguments +/// * `projection`: Reference to a ProjectionExec. +/// +/// # Returns +/// A `Result>` containing the downcasted columns or an error if downcasting fails. +/// fn downcast_projected_exprs_to_columns( projection: &ProjectionExec, ) -> Result> { @@ -3822,16 +3867,21 @@ fn downcast_projected_exprs_to_columns( Ok(columns.into_iter().cloned().collect()) } -/// Maps the indices of columns which are required from a projection node -/// such that the projection inserts the same requirements into its child. -/// -/// Example: +/// Maps the indices of required columns in a parent projection node to the corresponding indices in its child. /// /// 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" +/// +/// # Arguments +/// * `requirements`: Reference to a `HashSet` representing the parent's column requirements. +/// * `projection_columns`: Slice of `Column` representing the columns in the projection. +/// +/// # Returns +/// A `HashSet` with updated column indices reflecting the child's perspective. +/// fn map_parent_reqs_to_input_reqs( requirements: &HashSet, projection_columns: &[Column], @@ -3842,8 +3892,15 @@ fn map_parent_reqs_to_input_reqs( .collect::>() } -/// Compares the fields of a projection input schema with projection requirements. -/// If all input fields are required, then the function returns `true`. +/// Checks if all columns in the input schema are required by the projection. +/// +/// # Arguments +/// * `input_columns`: Reference to a `HashSet` representing the input columns. +/// * `projection_requires`: Reference to a `HashSet` representing the projection requirements. +/// +/// # Returns +/// `true` if all input columns are required, otherwise `false`. +/// fn all_input_columns_required( input_columns: &HashSet, projection_requires: &HashSet, @@ -3853,10 +3910,18 @@ fn all_input_columns_required( .all(|input_column| projection_requires.contains(input_column)) } -/// When we remove a redundant projection from a plan, parent operators are informed -/// about the index changes caused by the removal of this projection. This function -/// iterates the columns and records the changes for all of them after the removal. -/// If a change is observed, the old and new index values are inserted into a hashmap. +/// Calculates the index changes of columns after the removal of a projection. +/// +/// This function iterates through the columns and records the changes in their indices +/// 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 and new index values in a hashmap. +/// +/// # Arguments +/// * `columns` - A set of columns before the projection is removed. +/// * `projection_columns` - A slice of columns as they appear in the projection. +/// +/// # Returns +/// A `HashMap` where the key is the original column and the value is the column with updated index. fn index_changes_after_projection_removal( columns: HashSet, projection_columns: &[Column], @@ -3874,6 +3939,19 @@ fn index_changes_after_projection_removal( .collect() } +/// Rewrites a projection execution plan with updated column indices. +/// +/// This function updates the column indices in a projection based on a provided mapping. +/// It creates a new `ProjectionExec` with the updated expressions. +/// +/// # Arguments +/// * `projection` - The original projection execution plan. +/// * `input_plan` - The input execution plan on which the projection is applied. +/// * `mapping` - A hashmap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `ProjectionExec` wrapped in an `Arc`. +/// fn rewrite_projection( projection: &ProjectionExec, input_plan: Arc, @@ -3890,6 +3968,18 @@ fn rewrite_projection( .map(|plan| Arc::new(plan) as _) } +/// Rewrites a filter execution plan with updated column indices. +/// +/// This function updates the column indices in a filter's predicate based on a provided mapping. +/// It creates a new `FilterExec` with the updated predicate. +/// +/// # Arguments +/// * `predicate` - The predicate expression of the filter. +/// * `input_plan` - The input execution plan on which the filter is applied. +/// * `mapping` - A hashmap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `FilterExec` wrapped in an `Arc`. fn rewrite_filter( predicate: &Arc, input_plan: Arc, @@ -3899,6 +3989,18 @@ fn rewrite_filter( .map(|plan| Arc::new(plan) as _) } +/// Rewrites a repartition execution plan with updated column indices. +/// +/// Updates the partitioning expressions in a repartition plan based on the provided column index mappings. +/// Supports updating the `Partitioning::Hash` variant of partitioning. +/// +/// # Arguments +/// * `partitioning` - The original partitioning strategy. +/// * `input_plan` - The input execution plan on which repartitioning is applied. +/// * `mapping` - A hashmap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `RepartitionExec` wrapped in an `Arc`. fn rewrite_repartition( partitioning: &Partitioning, input_plan: Arc, @@ -3913,6 +4015,18 @@ fn rewrite_repartition( RepartitionExec::try_new(input_plan, new_partitioning).map(|plan| Arc::new(plan) as _) } +/// Rewrites a sort execution plan with updated column indices. +/// +/// This function updates the column indices in a sort's expressions based on a provided mapping. +/// It creates a new `SortExec` with the updated expressions. +/// +/// # Arguments +/// * `sort` - The original sort execution plan. +/// * `input_plan` - The input execution plan on which sorting is applied. +/// * `mapping` - A hashmap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `SortExec` wrapped in an `Arc`. fn rewrite_sort( sort: &SortExec, input_plan: Arc, @@ -3926,6 +4040,17 @@ fn rewrite_sort( ) as _) } +/// Rewrites a sort preserving merge execution plan with updated column indices. +/// +/// Updates the sort expressions in a sort preserving merge plan based on the provided column index mappings. +/// +/// # Arguments +/// * `sort` - The original `SortPreservingMergeExec` plan. +/// * `input_plan` - The input execution plan to which the sort preserving merge is applied. +/// * `mapping` - A hashmap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `SortPreservingMergeExec` wrapped in an `Arc`. fn rewrite_sort_preserving_merge( sort: &SortPreservingMergeExec, input_plan: Arc, @@ -3937,6 +4062,21 @@ fn rewrite_sort_preserving_merge( ) as _) } +/// Rewrites a hash join execution plan with updated column indices. +/// +/// Updates the join conditions and filter expressions in a hash join plan based on provided column index mappings +/// for both left and right input plans. +/// +/// # Arguments +/// * `hj` - The original `HashJoinExec` plan. +/// * `left_input_plan` - The left input execution plan. +/// * `right_input_plan` - The right input execution plan. +/// * `left_mapping` - A hashmap with old and new column index mappings for the left input. +/// * `right_mapping` - A hashmap with old and new column index mappings for the right input. +/// * `left_size` - The size of the left input columns set. +/// +/// # Returns +/// A `Result` containing the new `HashJoinExec` wrapped in an `Arc`. fn rewrite_hash_join( hj: &HashJoinExec, left_input_plan: Arc, @@ -4039,6 +4179,20 @@ fn rewrite_nested_loop_join( .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. +/// +/// # Arguments +/// * `smj`: The original SortMergeJoinExec to be rewritten. +/// * `left_input_plan`: The updated execution plan for the left input. +/// * `right_input_plan`: The updated execution plan for the right input. +/// * `left_mapping`: Column mapping for the left input. +/// * `right_mapping`: Column mapping for the right input. +/// * `left_size`: The size of the left input, necessary for index calculations. +/// +/// # Returns +/// A Result containing the rewritten execution plan as an Arc, or an error on failure. fn rewrite_sort_merge_join( smj: &SortMergeJoinExec, left_input_plan: Arc, @@ -4092,6 +4246,20 @@ fn rewrite_sort_merge_join( .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. +/// +/// # Arguments +/// * `shj`: The SymmetricHashJoinExec to be modified. +/// * `left_input_plan`: New execution plan for the left side. +/// * `right_input_plan`: New execution plan for the right side. +/// * `left_mapping`: Mapping for updating left side columns. +/// * `right_mapping`: Mapping for updating right side columns. +/// * `left_size`: Size of the left input for index adjustments. +/// +/// # Returns +/// A Result containing the updated execution plan within an Arc, or an error if the operation fails. fn rewrite_symmetric_hash_join( shj: &SymmetricHashJoinExec, left_input_plan: Arc, @@ -4148,6 +4316,17 @@ fn rewrite_symmetric_hash_join( .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. +/// +/// # Arguments +/// * `agg`: The original AggregateExec to be rewritten. +/// * `input_plan`: The new execution plan to use as input. +/// * `mapping`: A mapping from old to new columns. +/// +/// # Returns +/// A Result that either contains an Option with the new execution plan wrapped in an Arc, or None if no rewriting is possible, along with an error on failure. fn rewrite_aggregate( agg: &AggregateExec, input_plan: Arc, @@ -4192,6 +4371,17 @@ fn rewrite_aggregate( .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. +/// +/// # Arguments +/// * `w_agg`: The WindowAggExec to be updated. +/// * `input_plan`: The new execution plan to be used. +/// * `mapping`: Column mapping for updating window expressions. +/// +/// # Returns +/// A Result containing either an Option with the updated execution plan in an Arc, or None if rewriting isn't feasible, and an error on failure. fn rewrite_window_aggregate( w_agg: &WindowAggExec, input_plan: Arc, @@ -4208,6 +4398,17 @@ fn rewrite_window_aggregate( .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. +/// +/// # Arguments +/// * `bw_agg`: The original BoundedWindowAggExec to be rewritten. +/// * `input_plan`: The new execution plan to use. +/// * `mapping`: Mapping for updating window expressions. +/// +/// # Returns +/// A Result containing an Option with the new execution plan wrapped in an Arc, or None if the rewrite is not possible, and an error on failure. fn rewrite_bounded_window_aggregate( bw_agg: &BoundedWindowAggExec, input_plan: Arc, From 1bb2d7e0c3297fda807f191dc79fe8321220d5b9 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 1 Mar 2024 13:18:34 +0300 Subject: [PATCH 20/85] Clone's and unwrap's are removed --- .../optimize_projections.rs | 715 +++++++++--------- 1 file changed, 373 insertions(+), 342 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 4093a990aaa1..f9c0f4c89ea3 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -64,7 +64,6 @@ use crate::physical_plan::ExecutionPlan; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::DataFusionError; use datafusion_common::{JoinSide, JoinType}; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::collect_columns; @@ -73,10 +72,11 @@ use datafusion_physical_expr::{ AggregateExpr, LexOrdering, Partitioning, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, }; -use datafusion_physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; +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::get_plan_string; use datafusion_physical_plan::insert::FileSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, @@ -154,11 +154,10 @@ impl ProjectionOptimizer { /// 4. The projection can be embedded into the source. /// If none of them is possible, it remains unchanged. fn optimize_projections(mut self) -> Result { - let projection_input = self.plan.children(); - let projection_input = projection_input[0].as_any(); + let projection_input = &self.plan.children()[0]; // We first need to check having 2 sequential projections in case of merging them. - if projection_input.is::() { + if projection_input.as_any().is::() { self = match self.try_unifying_projections()? { Transformed::Yes(unified_plans) => { // We need to re-run the rule on the new node since it may need further optimizations. @@ -171,7 +170,7 @@ impl ProjectionOptimizer { // The projection can be removed. To avoid making unnecessary operations, // try_remove should be called before try_narrow. - self = match self.try_remove_projection() { + self = match self.try_remove_projection()? { Transformed::Yes(removed) => { // We need to re-run the rule on the current node. It is // a new plan node and may need optimizations for sure. @@ -189,7 +188,7 @@ impl ProjectionOptimizer { }; // Source providers: - if projection_input.is::() { + if projection_input.as_any().is::() { self = match self.try_projected_csv() { Transformed::Yes(new_csv) => return Ok(new_csv), Transformed::No(no_change) => no_change, @@ -198,27 +197,36 @@ impl ProjectionOptimizer { // If none of them possible, we will continue to next node. Output requirements // of the projection in terms of projection input are inserted to child node. - let projection_plan = - self.plan.as_any().downcast_ref::().unwrap(); - self.children_nodes[0].required_columns = self - .required_columns - .iter() - .flat_map(|e| collect_columns(&projection_plan.expr()[e.index()].0)) - .collect::>(); + self.children_nodes[0].required_columns = if let Some(projection_plan) = + self.plan.as_any().downcast_ref::() + { + self.required_columns + .iter() + .flat_map(|e| collect_columns(&projection_plan.expr()[e.index()].0)) + .collect::>() + } 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_unifying_projections(mut self) -> Result> { // These are known to be a ProjectionExec. - let projection = self.plan.as_any().downcast_ref::().unwrap(); - let child_projection = self.children_nodes[0] + 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::() - .unwrap(); + else { + return Ok(Transformed::No(self)); + }; - if caching_projections(projection, child_projection) { + if caching_projections(projection, child_projection)? { return Ok(Transformed::No(self)); } @@ -249,18 +257,18 @@ impl ProjectionOptimizer { /// the projection can be safely removed: /// 1. Projection must have all column expressions without aliases. /// 2. Projection input is fully required by the projection output requirements. - fn try_remove_projection(mut self) -> Transformed { + fn try_remove_projection(mut self) -> Result> { // It must be a projection - let projection_exec = - self.plan.as_any().downcast_ref::().unwrap(); - // The projection must have all column expressions without aliases. - if !all_alias_free_columns(projection_exec.expr()) { - return Transformed::No(self); - } + let Some(projection_exec) = self.plan.as_any().downcast_ref::() + else { + return Ok(Transformed::No(self)); + }; - // The expressions are known to be all columns. - let projection_columns = - downcast_projected_exprs_to_columns(projection_exec).unwrap(); + // The projection must have all column expressions without aliases. + let Some(projection_columns) = 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 = @@ -274,14 +282,14 @@ impl ProjectionOptimizer { &projection_columns, ); let new_current_node = self.children_nodes.swap_remove(0); - Transformed::Yes(ProjectionOptimizer { + Ok(Transformed::Yes(ProjectionOptimizer { plan: new_current_node.plan, required_columns: projection_requires, schema_mapping, children_nodes: new_current_node.children_nodes, - }) + })) } else { - Transformed::No(self) + Ok(Transformed::No(self)) } } @@ -289,8 +297,10 @@ impl ProjectionOptimizer { /// rewritten with a narrower schema, it is done so. Otherwise, it returns `None`. fn try_narrow_projection(self) -> Result> { // It must be a projection. - let projection_exec = - self.plan.as_any().downcast_ref::().unwrap(); + let Some(projection_exec) = self.plan.as_any().downcast_ref::() + else { + return Ok(Transformed::No(self)); + }; let requirement_map = self.analyze_requirements(); let (used_columns, unused_columns) = split_column_requirements(&requirement_map); @@ -347,47 +357,43 @@ impl ProjectionOptimizer { /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. fn try_projected_csv(self) -> Transformed { // These plans are known. - let projection = self.plan.as_any().downcast_ref::().unwrap(); - let csv = projection - .input() - .as_any() - .downcast_ref::() - .unwrap(); + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Transformed::No(self); + }; + let Some(csv) = projection.input().as_any().downcast_ref::() else { + return Transformed::No(self); + }; // If there is any non-column or alias-carrier expression, Projection should not be removed. // This process can be moved into CsvExec, but it could be a conflict of their responsibility. - if all_alias_free_columns(projection.expr()) { - let mut file_scan = csv.base_config().clone(); - let projection_columns = projection - .expr() - .iter() - .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap()) - .collect::>(); + let Some(projection_columns) = collect_alias_free_columns(projection.expr()) + else { + return Transformed::No(self); + }; - let new_projections = new_projections_for_columns( - &projection_columns, - &file_scan - .projection - .unwrap_or((0..csv.schema().fields().len()).collect()), - ); + let mut file_scan = csv.base_config().clone(); - file_scan.projection = Some(new_projections); - - Transformed::Yes(ProjectionOptimizer { - plan: Arc::new(CsvExec::new( - file_scan, - csv.has_header(), - csv.delimiter(), - csv.quote(), - csv.escape(), - csv.file_compression_type, - )) as _, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), // Sources cannot have a mapping. - children_nodes: vec![], - }) - } else { - Transformed::No(self) - } + let new_projections = new_projections_for_columns( + &projection_columns, + &file_scan + .projection + .unwrap_or((0..csv.schema().fields().len()).collect()), + ); + + file_scan.projection = Some(new_projections); + + Transformed::Yes(ProjectionOptimizer { + plan: Arc::new(CsvExec::new( + file_scan, + csv.has_header(), + csv.delimiter(), + csv.quote(), + csv.escape(), + csv.file_compression_type, + )) as _, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), // Sources cannot have a mapping. + children_nodes: vec![], + }) } /// If the node plan can be rewritten with a narrower schema, a projection is inserted @@ -399,15 +405,8 @@ impl ProjectionOptimizer { fn try_projection_insertion(mut self) -> Result { let plan = self.plan.clone(); - if let Some(_projection) = plan.as_any().downcast_ref::() { - panic!( - "\"try_projection_insertion\" subrule cannot be used on ProjectionExec's." - ); - } else if let Some(_csv) = plan.as_any().downcast_ref::() { - panic!("\"try_projection_insertion\" subrule cannot be used on plans with no child.") - } // These plans preserve the input schema, and do not add new requirements. - else if let Some(coal_b) = plan.as_any().downcast_ref::() { + if let Some(coal_b) = plan.as_any().downcast_ref::() { self = self.try_insert_below_coalesce_batches(coal_b)?; } else if plan .as_any() @@ -928,12 +927,11 @@ impl ProjectionOptimizer { } // Left child needs a projection. (false, true) => { + let required_columns = self.required_columns.clone(); let mut right_child = self.children_nodes.swap_remove(1); - let (new_left_child, left_schema_mapping) = self - .clone() - .insert_projection_below_single_child(analyzed_join_left, 0)?; - right_child.required_columns = self - .required_columns + let (new_left_child, 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)) @@ -952,10 +950,10 @@ impl ProjectionOptimizer { } // Right child needs a projection. (true, false) => { - let mut left_child = self.children_nodes[0].clone(); - let (new_right_child, mut right_schema_mapping) = self - .clone() - .insert_projection_below_single_child(analyzed_join_right, 1)?; + let required_columns = self.required_columns.clone(); + let mut left_child = self.children_nodes.swap_remove(0); + let (new_right_child, mut right_schema_mapping) = + self.insert_projection_below_right_child(analyzed_join_right)?; right_schema_mapping = right_schema_mapping .into_iter() .map(|(old, new)| { @@ -965,8 +963,7 @@ impl ProjectionOptimizer { ) }) .collect(); - left_child.required_columns = self - .required_columns + left_child.required_columns = required_columns .iter() .filter(|col| col.index() < left_size) .cloned() @@ -1089,6 +1086,7 @@ impl ProjectionOptimizer { } } (false, true) => { + let required_columns = self.required_columns.clone(); let mut right_child = self.children_nodes.swap_remove(1); let new_on = update_equivalence_conditions( hj.on(), @@ -1101,12 +1099,8 @@ impl ProjectionOptimizer { &HashMap::new(), ); let (new_left_child, mut left_schema_mapping) = - self.clone().insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; - right_child.required_columns = self - .required_columns + 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)) @@ -1145,7 +1139,8 @@ impl ProjectionOptimizer { } } (true, false) => { - let mut left_child = self.children_nodes[0].clone(); + let mut left_child = self.children_nodes.swap_remove(0); + let required_columns = self.required_columns.clone(); let new_on = update_equivalence_conditions( hj.on(), &HashMap::new(), @@ -1156,11 +1151,8 @@ impl ProjectionOptimizer { &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, mut right_schema_mapping) = - self.clone().insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + let (new_right_child, mut right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; right_schema_mapping = right_schema_mapping .into_iter() .map(|(old, new)| { @@ -1170,8 +1162,7 @@ impl ProjectionOptimizer { ) }) .collect(); - left_child.required_columns = self - .required_columns + left_child.required_columns = required_columns .iter() .filter(|col| col.index() < left_size) .cloned() @@ -1219,10 +1210,7 @@ impl ProjectionOptimizer { ); let (new_left_child, left_schema_mapping) = - self.clone().insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; + self.insert_projection_below_left_child(analyzed_join_left)?; let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), @@ -1262,7 +1250,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes[0].clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( hj.on(), &HashMap::new(), @@ -1275,10 +1263,7 @@ impl ProjectionOptimizer { ); let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + .insert_projection_below_right_child(analyzed_join_right)?; let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), @@ -1374,6 +1359,7 @@ impl ProjectionOptimizer { } } (false, true) => { + let required_columns = self.required_columns.clone(); let mut right_child = self.children_nodes.swap_remove(1); let new_filter = update_non_equivalence_conditions( nlj.filter(), @@ -1381,12 +1367,8 @@ impl ProjectionOptimizer { &HashMap::new(), ); let (new_left_child, mut left_schema_mapping) = - self.clone().insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; - right_child.required_columns = self - .required_columns + 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)) @@ -1422,17 +1404,15 @@ impl ProjectionOptimizer { } } (true, false) => { - let mut left_child = self.children_nodes[0].clone(); + let required_columns = self.required_columns.clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_filter = update_non_equivalence_conditions( nlj.filter(), &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, mut right_schema_mapping) = - self.clone().insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + let (new_right_child, mut right_schema_mapping) = self + .insert_projection_below_right_child(analyzed_join_right)?; right_schema_mapping = right_schema_mapping .into_iter() .map(|(old, new)| { @@ -1442,8 +1422,7 @@ impl ProjectionOptimizer { ) }) .collect(); - left_child.required_columns = self - .required_columns + left_child.required_columns = required_columns .iter() .filter(|col| col.index() < left_size) .cloned() @@ -1520,17 +1499,14 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes[0].clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_filter = update_non_equivalence_conditions( nlj.filter(), &HashMap::new(), &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + .insert_projection_below_right_child(analyzed_join_right)?; let plan = Arc::new(NestedLoopJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1664,7 +1640,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes[0].clone(); + let left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( smj.on(), &HashMap::new(), @@ -1676,10 +1652,7 @@ impl ProjectionOptimizer { &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + .insert_projection_below_right_child(analyzed_join_right)?; let plan = Arc::new(SortMergeJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1763,7 +1736,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes[0].clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( smj.on(), &HashMap::new(), @@ -1775,10 +1748,7 @@ impl ProjectionOptimizer { &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + .insert_projection_below_right_child(analyzed_join_right)?; let plan = Arc::new(SortMergeJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1921,7 +1891,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes[0].clone(); + let left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( shj.on(), &HashMap::new(), @@ -1933,10 +1903,7 @@ impl ProjectionOptimizer { &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + .insert_projection_below_right_child(analyzed_join_right)?; let plan = Arc::new(SymmetricHashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -2024,7 +1991,7 @@ impl ProjectionOptimizer { JoinType::RightAnti | JoinType::RightSemi => { match all_columns_required(&analyzed_join_right) { false => { - let mut left_child = self.children_nodes[0].clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( shj.on(), &HashMap::new(), @@ -2036,10 +2003,7 @@ impl ProjectionOptimizer { &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_right, - 1, - )?; + .insert_projection_below_right_child(analyzed_join_right)?; let plan = Arc::new(SymmetricHashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -2133,23 +2097,23 @@ impl ProjectionOptimizer { self.required_columns = HashSet::new(); } else { match agg.mode() { - datafusion_physical_plan::aggregates::AggregateMode::Final - | datafusion_physical_plan::aggregates::AggregateMode::FinalPartitioned => - { + AggregateMode::Final | AggregateMode::FinalPartitioned => { let mut group_expr_len = agg.group_expr().expr().iter().count(); let aggr_columns = agg .aggr_expr() .iter() .flat_map(|e| { - e.state_fields() - .unwrap() - .iter() - .map(|field| { - group_expr_len += 1; - Column::new(field.name(), group_expr_len - 1) - }) - .collect::>() + e.state_fields().map(|field| { + field + .iter() + .map(|field| { + group_expr_len += 1; + Column::new(field.name(), group_expr_len - 1) + }) + .collect::>() + }) }) + .flatten() .collect::>(); let group_columns = agg .group_expr() @@ -2227,11 +2191,27 @@ impl ProjectionOptimizer { w_agg.input().schema().fields().len(), &requirement_map, ) { - let (new_child, schema_mapping, window_usage) = self - .clone() - .insert_projection_below_window(w_agg, requirement_map)?; + if w_agg + .window_expr() + .iter() + .any(|expr| expr.clone().with_new_expressions(vec![]).is_none()) + { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < w_agg.schema().fields().len() + - w_agg.window_expr().len() + }) + .cloned() + .collect(); + return Ok(self); + } + let (new_child, schema_mapping, window_usage) = + self.insert_projection_below_window(w_agg, requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. - let new_window_exprs = w_agg + let Some(new_window_exprs) = w_agg .window_expr() .iter() .zip(window_usage.clone()) @@ -2239,12 +2219,16 @@ impl ProjectionOptimizer { .map(|(window_expr, (_window_col, _usage))| { let new_exprs = update_expressions( &window_expr.expressions(), - &schema_mapping, + &HashMap::new(), ); window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() - .unwrap(); + else { + return Err(datafusion_common::DataFusionError::Internal( + "".to_string(), + )); + }; let new_keys = w_agg .partition_keys @@ -2314,11 +2298,27 @@ impl ProjectionOptimizer { bw_agg.input().schema().fields().len(), &requirement_map, ) { - let (new_child, schema_mapping, window_usage) = self - .clone() - .insert_projection_below_bounded_window(bw_agg, requirement_map)?; + if bw_agg + .window_expr() + .iter() + .any(|expr| expr.clone().with_new_expressions(vec![]).is_none()) + { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .filter(|col| { + col.index() + < bw_agg.schema().fields().len() + - bw_agg.window_expr().len() + }) + .cloned() + .collect(); + return Ok(self); + } + let (new_child, schema_mapping, window_usage) = + self.insert_projection_below_bounded_window(bw_agg, requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. - let new_window_exprs = bw_agg + let Some(new_window_exprs) = bw_agg .window_expr() .iter() .zip(window_usage.clone()) @@ -2331,7 +2331,11 @@ impl ProjectionOptimizer { window_expr.clone().with_new_expressions(new_exprs) }) .collect::>>() - .unwrap(); + else { + return Err(datafusion_common::DataFusionError::Internal( + "".to_string(), + )); + }; let new_keys = bw_agg .partition_keys @@ -2403,10 +2407,7 @@ impl ProjectionOptimizer { // 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) = split_column_requirements(&requirement_map); - let mut projected_exprs = convert_projection_exprs(used_columns); - projected_exprs.sort_by_key(|(expr, _alias)| { - expr.as_any().downcast_ref::().unwrap().index() - }); + let projected_exprs = convert_projection_exprs(used_columns); let inserted_projection = Arc::new(ProjectionExec::try_new( projected_exprs, self.plan.children()[0].clone(), @@ -2434,10 +2435,8 @@ impl ProjectionOptimizer { // 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) = split_column_requirements(&requirement_map); - let mut projected_exprs = convert_projection_exprs(used_columns); - projected_exprs.sort_by_key(|(expr, _alias)| { - expr.as_any().downcast_ref::().unwrap().index() - }); + let projected_exprs = convert_projection_exprs(used_columns); + let inserted_projections = self .plan .children() @@ -2475,7 +2474,7 @@ impl ProjectionOptimizer { /// Single child version of `insert_projection` for joins. fn insert_projection_below_single_child( - self, + mut self, requirement_map_left: ColumnRequirements, children_index: usize, ) -> Result<(Self, HashMap)> { @@ -2483,30 +2482,116 @@ impl ProjectionOptimizer { // and also collect the unused columns to store the index changes after removal of some columns. let (used_columns, unused_columns) = split_column_requirements(&requirement_map_left); - let mut projected_exprs = convert_projection_exprs(used_columns); - projected_exprs.sort_by_key(|(expr, _alias)| { - expr.as_any().downcast_ref::().unwrap().index() - }); - let child_plan = self.plan.children().remove(children_index); + 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 = projected_exprs - .iter() - .map(|(expr, _alias)| expr.as_any().downcast_ref::().unwrap().clone()) - .collect::>(); + let required_columns = collect_columns_in_plan_schema(&inserted_projection); + let inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(children_index)], + }; + Ok((inserted_projection, new_mapping)) + } - let new_mapping = calculate_column_mapping(&required_columns, &unused_columns); + /// Single child version of `insert_projection` for joins. + #[allow(clippy::type_complexity)] + fn insert_projection_below_multi_child( + mut self, + requirement_map_left: ColumnRequirements, + requirement_map_right: ColumnRequirements, + ) -> Result<(Self, Self, HashMap, HashMap)> { + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let (used_columns, unused_columns) = + split_column_requirements(&requirement_map_left); + let child_plan = self.plan.children().remove(0); + let new_left_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 left_inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + + let (used_columns, unused_columns) = + split_column_requirements(&requirement_map_right); + let child_plan = self.plan.children().remove(1); + let new_right_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 right_inserted_projection = ProjectionOptimizer { + plan: inserted_projection, + required_columns, + schema_mapping: HashMap::new(), + children_nodes: vec![self.children_nodes.swap_remove(0)], + }; + Ok(( + left_inserted_projection, + right_inserted_projection, + new_left_mapping, + new_right_mapping, + )) + } + /// Left child version of `insert_projection` for joins. + fn insert_projection_below_left_child( + mut self, + requirement_map_left: ColumnRequirements, + ) -> Result<(Self, HashMap)> { + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let (used_columns, unused_columns) = + split_column_requirements(&requirement_map_left); + let child_plan = self.plan.children().remove(0); + 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: HashMap::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, HashMap)> { + // During the iteration, we construct the ProjectionExec with required columns as the new child, + // and also collect the unused columns to store the index changes after removal of some columns. + let (used_columns, unused_columns) = + split_column_requirements(&requirement_map_right); + let child_plan = self.plan.children().remove(1); + 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: HashMap::new(), - children_nodes: vec![self.children_nodes[children_index].clone()], + children_nodes: vec![self.children_nodes.swap_remove(0)], }; Ok((inserted_projection, new_mapping)) } @@ -2519,11 +2604,15 @@ impl ProjectionOptimizer { requirement_map_right: ColumnRequirements, ) -> Result<(Self, Self, HashMap)> { let original_right = self.children_nodes[1].plan.clone(); - let (new_left_child, mut left_schema_mapping) = self - .clone() - .insert_projection_below_single_child(requirement_map_left, 0)?; - let (new_right_child, right_schema_mapping) = - self.insert_projection_below_single_child(requirement_map_right, 1)?; + let ( + new_left_child, + new_right_child, + mut left_schema_mapping, + right_schema_mapping, + ) = self.insert_projection_below_multi_child( + requirement_map_left, + requirement_map_right, + )?; let new_left_size = new_left_child.plan.schema().fields().len(); // left_schema_mapping does not need to be change, but it is updated with @@ -2535,18 +2624,9 @@ impl ProjectionOptimizer { .iter() .enumerate() .filter(|(idx, field)| { - let right_projection = new_right_child - .plan - .as_any() - .downcast_ref::() - .unwrap() - .expr() - .iter() - .map(|(expr, _alias)| { - expr.as_any().downcast_ref::().unwrap() - }) - .collect::>(); - right_projection.contains(&&Column::new(field.name(), *idx)) + let right_projection = + collect_columns_in_plan_schema(&new_right_child.plan); + right_projection.contains(&Column::new(field.name(), *idx)) }) { left_schema_mapping.insert( @@ -2756,10 +2836,10 @@ impl ProjectionOptimizer { let right_mapping = all_mappings.swap_remove(0); let new_mapping = left_mapping .iter() - .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .map(|(initial, new)| (initial.clone(), new.clone())) .chain(right_mapping.iter().map(|(initial, new)| { ( - Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(initial.name(), initial.index() + left_size), Column::new(new.name(), new.index() + left_size), ) })) @@ -2799,10 +2879,10 @@ impl ProjectionOptimizer { let right_mapping = all_mappings.swap_remove(0); let new_mapping = left_mapping .iter() - .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .map(|(initial, new)| (initial.clone(), new.clone())) .chain(right_mapping.iter().map(|(initial, new)| { ( - Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(initial.name(), initial.index() + left_size), Column::new(new.name(), new.index() + left_size), ) })) @@ -2841,10 +2921,10 @@ impl ProjectionOptimizer { let right_mapping = all_mappings.swap_remove(0); let new_mapping = left_mapping .iter() - .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .map(|(initial, new)| (initial.clone(), new.clone())) .chain(right_mapping.iter().map(|(initial, new)| { ( - Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(initial.name(), initial.index() + left_size), Column::new(new.name(), new.index() + left_size), ) })) @@ -2872,10 +2952,10 @@ impl ProjectionOptimizer { let right_mapping = all_mappings.swap_remove(0); let new_mapping = left_mapping .iter() - .map(|(initial, new)| (initial.clone(), new.clone())) // Clone the columns from left_mapping + .map(|(initial, new)| (initial.clone(), new.clone())) .chain(right_mapping.iter().map(|(initial, new)| { ( - Column::new(initial.name(), initial.index() + left_size), // Create new Column instances for right_mapping + Column::new(initial.name(), initial.index() + left_size), Column::new(new.name(), new.index() + left_size), ) })) @@ -3022,65 +3102,54 @@ impl ProjectionOptimizer { }; // Is the projection really required? First, we need to // have all column expression in the projection for removal. - if all_alias_free_columns(projection.expr()) { - // Then, check if all columns in the input schema exist after - // the projection. If it is so, we can remove the projection - // since it does not provide any benefit. - let child_columns = collect_columns_in_plan_schema(projection.input()); - let projection_columns = projection - .expr() - .iter() - .map(|(expr, _alias)| { - // We have ensured all expressions are column. - expr.as_any().downcast_ref::().unwrap().clone() - }) - .collect::>(); - 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); + let Some(projection_columns) = collect_alias_free_columns(projection.expr()) + else { + return Ok(self); + }; - if self_mapping.is_empty() { - self.schema_mapping = projection - .expr() - .iter() - .enumerate() - .filter_map(|(idx, (col, _alias))| { - let new_column = - col.as_any().downcast_ref::().unwrap(); - if new_column.index() != idx { - Some(( - Column::new(new_column.name(), idx), - new_column.clone(), - )) - } else { - None - } - }) - .collect(); - } else { - self.schema_mapping = self_mapping - .into_iter() - .map(|(expected, updated)| { - ( - expected, - Column::new( - updated.name(), - projection_columns[updated.index()].index(), - ), - ) - }) - .collect() - } + // 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) @@ -3219,9 +3288,6 @@ impl PhysicalOptimizerRule for OptimizeProjections { // Ensure the final optimized plan satisfies the initial schema requirements. optimized = satisfy_initial_schema(optimized, initial_requirements)?; - // TODO: Remove this check to tests - crosscheck_helper(optimized.clone())?; - Ok(optimized.plan) } @@ -3234,24 +3300,6 @@ impl PhysicalOptimizerRule for OptimizeProjections { } } -// TODO: Remove this to tests -pub fn crosscheck_helper(context: ProjectionOptimizer) -> Result<()> { - context.transform_up(&|node| { - assert_eq!(node.children_nodes.len(), node.plan.children().len()); - if !node.children_nodes.is_empty() { - assert_eq!( - get_plan_string(&node.plan), - get_plan_string(&node.plan.clone().with_new_children( - node.children_nodes.iter().map(|c| c.plan.clone()).collect() - )?) - ); - } - Ok(Transformed::No(node)) - })?; - - Ok(()) -} - /// Ensures that the output schema `po` matches the `initial_requirements`. /// If the `schema_mapping` of `po` indicates that some columns have been re-mapped, /// a new projection is added to restore the initial column order and indices. @@ -3363,10 +3411,10 @@ fn window_agg_required( fn caching_projections( projection: &ProjectionExec, child_projection: &ProjectionExec, -) -> bool { +) -> Result { let mut column_ref_map: HashMap = HashMap::new(); // Collect the column references' usage in the parent projection. - projection.expr().iter().for_each(|(expr, _)| { + projection.expr().iter().try_for_each(|(expr, _)| { expr.apply(&mut |expr| { Ok({ if let Some(column) = expr.as_any().downcast_ref::() { @@ -3374,12 +3422,12 @@ fn caching_projections( } VisitRecursion::Continue }) - }) - .unwrap(); - }); - column_ref_map.iter().any(|(column, count)| { + })?; + Ok(()) as Result<()> + })?; + Ok(column_ref_map.iter().any(|(column, count)| { *count > 1 && !is_expr_trivial(&child_projection.expr()[column.index()].0) - }) + })) } /// Checks if the given expression is trivial. @@ -3389,21 +3437,27 @@ fn is_expr_trivial(expr: &Arc) -> bool { || expr.as_any().downcast_ref::().is_some() } -/// Given the expression set of a projection, checks if the projection causes -/// any renaming or constructs a non-`Column` physical expression. -fn all_alias_free_columns(exprs: &[(Arc, String)]) -> bool { - exprs.iter().all(|(expr, alias)| { - expr.as_any() - .downcast_ref::() - .map(|column| column.name() == alias) - .unwrap_or(false) - }) +/// 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 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) } /// Updates a source provider's projected columns according to the given /// projection operator's expressions. To use this function safely, one must /// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns(projection: &[&Column], source: &[usize]) -> Vec { +fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec { projection.iter().map(|col| source[col.index()]).collect() } @@ -3455,14 +3509,15 @@ fn split_column_requirements( fn convert_projection_exprs( columns: HashSet, ) -> Vec<(Arc, String)> { - let result = columns + 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::>(); - result + .collect() } #[derive(Debug, PartialEq)] @@ -3843,30 +3898,6 @@ fn removed_column_count( left_skipped_columns } -/// Downcast all expressions in a projection to their [`Column`] parts, returning an error if any downcast fails. -/// -/// # Arguments -/// * `projection`: Reference to a ProjectionExec. -/// -/// # Returns -/// A `Result>` containing the downcasted columns or an error if downcasting fails. -/// -fn downcast_projected_exprs_to_columns( - projection: &ProjectionExec, -) -> Result> { - let Some(columns) = projection - .expr() - .iter() - .map(|(expr, _)| expr.as_any().downcast_ref::()) - .collect::>>() - else { - return Err(DataFusionError::Internal("PhysicalExpr which is tried to be downcasted is not a Column. - `downcast_projected_exprs_to_columns` can be used after the check of `all_alias_free_columns`".to_string())); - }; - - Ok(columns.into_iter().cloned().collect()) -} - /// Maps the indices of required columns in a parent projection node to the corresponding indices in its child. /// /// Projection is required to have columns at "@0:a - @1:b - @2:c" From c8f39fe17c6b713fbc39a7e0f65614111b5cf6cd Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 1 Mar 2024 13:27:49 +0300 Subject: [PATCH 21/85] Update optimize_projections.rs --- datafusion/core/src/physical_optimizer/optimize_projections.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index f9c0f4c89ea3..3a2abc40609e 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -194,6 +194,7 @@ impl ProjectionOptimizer { Transformed::No(no_change) => no_change, } } + // TODO: Other source execs can be implemented here if projection can be applied within the scope of 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. @@ -578,7 +579,7 @@ impl ProjectionOptimizer { ) -> Result { // GlobalLimitExec does not change requirements. We can directly check whether there is a redundancy. let requirement_map = self.analyze_requirements(); - if true { + if all_columns_required(&requirement_map) { // if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); From aa296dc0da4bca310c0e11ce92695a3ce7fdf23e Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 1 Mar 2024 14:07:16 +0300 Subject: [PATCH 22/85] add yaml --- .github/workflows/docs.yaml | 64 +++++++++++++++++++ .../optimize_projections.rs | 1 + 2 files changed, 65 insertions(+) create mode 100644 .github/workflows/docs.yaml diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml new file mode 100644 index 000000000000..660513ca0ec9 --- /dev/null +++ b/.github/workflows/docs.yaml @@ -0,0 +1,64 @@ +on: + push: + branches: + - main + paths: + - .asf.yaml + - .github/workflows/docs.yaml + - docs/** + +name: Deploy DataFusion site + +jobs: + build-docs: + name: Build docs + runs-on: ubuntu-latest + steps: + - name: Checkout docs sources + uses: actions/checkout@v4 + + - name: Checkout asf-site branch + uses: actions/checkout@v4 + with: + ref: asf-site + path: asf-site + + - name: Setup Python + uses: actions/setup-python@v5 + with: + python-version: "3.10" + + - name: Install dependencies + run: | + set -x + python3 -m venv venv + source venv/bin/activate + pip install -r docs/requirements.txt + + - name: Build docs + run: | + set -x + source venv/bin/activate + cd docs + ./build.sh + + - name: Copy & push the generated HTML + run: | + set -x + cd asf-site/ + rsync \ + -a \ + --delete \ + --exclude '/.git/' \ + ../docs/build/html/ \ + ./ + cp ../.asf.yaml . + touch .nojekyll + git status --porcelain + if [ "$(git status --porcelain)" != "" ]; then + git config user.name "github-actions[bot]" + git config user.email "github-actions[bot]@users.noreply.github.com" + git add --all + git commit -m 'Publish built docs triggered by ${{ github.sha }}' + git push || git push --force + fi diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 3a2abc40609e..b76eebad98dc 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3280,6 +3280,7 @@ impl PhysicalOptimizerRule for OptimizeProjections { // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); let mut optimizer = ProjectionOptimizer::new_default(plan); + // Insert the initial requirements to the root node, and run the rule. optimizer.required_columns = initial_requirements.clone(); let mut optimized = optimizer.transform_down(&|o| { From 8b467753da938a3951ed581fef1b94a84b55f43a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 1 Mar 2024 14:52:30 +0300 Subject: [PATCH 23/85] projection update --- .github/workflows/docs.yaml | 2 +- .../src/equivalence/projection.rs | 34 ++++++++++--------- datafusion/physical-plan/src/projection.rs | 27 ++++----------- 3 files changed, 26 insertions(+), 37 deletions(-) diff --git a/.github/workflows/docs.yaml b/.github/workflows/docs.yaml index 660513ca0ec9..ab6a615ab60b 100644 --- a/.github/workflows/docs.yaml +++ b/.github/workflows/docs.yaml @@ -41,7 +41,7 @@ jobs: source venv/bin/activate cd docs ./build.sh - + - name: Copy & push the generated HTML run: | set -x diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 0f92b2c2f431..fbd22cc5ac5a 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -21,6 +21,7 @@ use crate::expressions::Column; use crate::PhysicalExpr; use arrow::datatypes::SchemaRef; +use arrow_schema::Schema; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; @@ -56,22 +57,7 @@ impl ProjectionMapping { .enumerate() .map(|(expr_idx, (expression, name))| { 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 - // doesn't match. This can cause problems, so we make sure - // that the expression name matches with the name in `input_schema`. - // Conceptually, `source_expr` and `expression` should be the same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - }) + Self::update_expr_with_input_schema(expression.clone(), input_schema) .map(|source_expr| (source_expr, target_expr)) }) .collect::>>() @@ -104,6 +90,22 @@ impl ProjectionMapping { .find(|(source, _)| source.eq(expr)) .map(|(_, target)| target.clone()) } + + /// This function ensures that an expression has the same name with the field of it in its input schema. + pub fn update_expr_with_input_schema( + expr: Arc, + input_schema: &Schema, + ) -> Result> { + expr.transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => { + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + }) + } } #[cfg(test)] diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 1ae6851a602e..506e115b1e79 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -71,26 +71,13 @@ impl ProjectionExec { ) -> Result { let input_schema = input.schema(); expr = expr - .iter() - .enumerate() - .map(|(_expr_idx, (expression, name))| { - expression - .clone() - .transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => { - // Sometimes, an expression and its name in the input_schema - // doesn't match. This can cause problems, so we make sure - // that the expression name matches with the name in `input_schema`. - // Conceptually, `source_expr` and `expression` should be the same. - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - }) - .map(|source_expr| (source_expr, name.to_string())) + .into_iter() + .map(|(expression, name)| { + ProjectionMapping::update_expr_with_input_schema( + expression, + &input_schema, + ) + .map(|source_expr| (source_expr, name.to_string())) }) .collect::>>()?; From 4ccab8b7884059d05d1fd2ef72b7de24f85eab9a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 1 Mar 2024 16:06:31 +0300 Subject: [PATCH 24/85] Pushdown through limits --- .../optimize_projections.rs | 25 +-- datafusion/sqllogictest/test_files/window.slt | 164 +++++++++--------- 2 files changed, 98 insertions(+), 91 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index b76eebad98dc..6f703d59a48a 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3287,7 +3287,9 @@ impl PhysicalOptimizerRule for OptimizeProjections { o.adjust_node_with_requirements().map(Transformed::Yes) })?; - // Ensure the final optimized plan satisfies the initial schema requirements. + // 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 = satisfy_initial_schema(optimized, initial_requirements)?; Ok(optimized.plan) @@ -3306,7 +3308,7 @@ impl PhysicalOptimizerRule for OptimizeProjections { /// If the `schema_mapping` of `po` indicates that some columns have been re-mapped, /// a new projection is added to restore the initial column order and indices. fn satisfy_initial_schema( - po: ProjectionOptimizer, + mut po: ProjectionOptimizer, initial_requirements: HashSet, ) -> Result { if collect_columns_in_plan_schema(&po.plan) == initial_requirements @@ -3315,18 +3317,17 @@ fn satisfy_initial_schema( // The initial schema is already satisfied, no further action required. Ok(po) } else { - // Collect expressions for the final projection to match the initial requirements. - let mut initial_requirements_vec = + let mut initial_requirements_ordered = initial_requirements.clone().into_iter().collect_vec(); - initial_requirements_vec.sort_by_key(|expr| expr.index()); - let projected_exprs = initial_requirements_vec - .iter() + 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 column_index = po.schema_mapping.get(col).unwrap_or(col).index(); - let new_col = Arc::new(Column::new(col.name(), column_index)) - as Arc; - (new_col, col.name().to_string()) + let final_column = po.schema_mapping.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::>(); @@ -3337,7 +3338,7 @@ fn satisfy_initial_schema( // Return a new ProjectionOptimizer with the final projection, resetting the schema mapping. Ok(ProjectionOptimizer { plan: final_projection, - required_columns: initial_requirements, + required_columns: HashSet::new(), schema_mapping: HashMap::new(), // Reset schema mapping as we've now satisfied the initial schema children_nodes: vec![po], // Keep the original node as the child }) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3b6f682b3495..33086d041fdf 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1435,14 +1435,15 @@ Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregat --------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]] ----------TableScan: aggregate_test_100 projection=[c1, c2, c9] physical_plan -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] +ProjectionExec: expr=[c9@0 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@3 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@1 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn2] --GlobalLimitExec: skip=0, fetch=5 -----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)) }], mode=[Sorted] -------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] ---------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)) }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }], mode=[Sorted] -------------SortExec: expr=[c9@2 DESC,c1@0 DESC] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true +----ProjectionExec: expr=[c9@2 as c9, 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 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, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, 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 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] +------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)) }], mode=[Sorted] +--------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +----------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)) }], mode=[Sorted] +------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }], mode=[Sorted] +--------------SortExec: expr=[c9@2 DESC,c1@0 DESC] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true query IIII SELECT @@ -1517,23 +1518,24 @@ Projection: SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BET --------------------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]] ----------------------TableScan: null_cases projection=[c1, c2, c3] physical_plan -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] +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] --GlobalLimitExec: skip=0, fetch=5 -----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)) }, 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 }, 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)) }, 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)) }] -------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] ---------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 }], mode=[Sorted] -----------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST] -------------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 }], mode=[Sorted] ---------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC] -----------------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 }], mode=[Sorted] -------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC] ---------------------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)) }, 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)) }, 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)) }, 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 }] -----------------------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)) }, 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 }, 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)) }, 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)) }] -------------------------SortExec: expr=[c3@2 DESC NULLS LAST] ---------------------------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)) }, 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 }, 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)) }, 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)) }] -----------------------------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 }], mode=[Sorted] -------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST] ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true +----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] +------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)) }, 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 }, 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)) }, 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)) }] +--------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] +----------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 }], mode=[Sorted] +------------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST] +--------------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 }], mode=[Sorted] +----------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC] +------------------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 }], mode=[Sorted] +--------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC] +----------------------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)) }, 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)) }, 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)) }, 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 }] +------------------------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)) }, 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 }, 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)) }, 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)) }] +--------------------------SortExec: expr=[c3@2 DESC NULLS LAST] +----------------------------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)) }, 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 }, 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)) }, 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)) }] +------------------------------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 }], mode=[Sorted] +--------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST] +----------------------------------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,13 +1605,13 @@ Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregat ------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]] --------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -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] +ProjectionExec: expr=[c9@0 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@1 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] --GlobalLimitExec: skip=0, fetch=5 - ----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)) }], mode=[Sorted] -------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)) }], mode=[Sorted] ---------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true - +----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 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, 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 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] +------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)) }], mode=[Sorted] +--------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)) }], mode=[Sorted] +----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1647,12 +1649,13 @@ Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggr ------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]] --------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -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] +ProjectionExec: expr=[c9@0 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@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@1 as sum2] --GlobalLimitExec: skip=0, fetch=5 -----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)) }], mode=[Sorted] -------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)) }], mode=[Sorted] ---------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +----ProjectionExec: expr=[c9@1 as c9, 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 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, 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 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] +------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)) }], mode=[Sorted] +--------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)) }], mode=[Sorted] +----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1693,17 +1696,17 @@ Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregat ----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 ------------TableScan: aggregate_test_100 projection=[c2, c3, c4, c9] physical_plan -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] +ProjectionExec: expr=[c3@0 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@1 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@2 as sum2] --GlobalLimitExec: skip=0, fetch=5 -----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)) }] -------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] ---------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 }], mode=[Sorted] -----------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] ---------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] -----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true - +----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 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, 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 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] +------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)) }] +--------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] +----------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 }], mode=[Sorted] +------------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +--------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +----------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] +------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true query III SELECT c3, @@ -2137,17 +2140,18 @@ Projection: t1.c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NU --------------------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c8, aggregate_test_100.c9, aggregate_test_100.c1 AS c1_alias ----------------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] physical_plan -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] +ProjectionExec: expr=[c9@0 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@2 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@4 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@1 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@3 as sum4] --GlobalLimitExec: skip=0, fetch=5 -----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)) }], mode=[Sorted] -------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] ---------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)) }] -----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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)) }], mode=[Sorted] ---------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }] -----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] -------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +----ProjectionExec: expr=[c9@1 as c9, 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 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@4 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@5 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, 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 SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +------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)) }], mode=[Sorted] +--------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] +----------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)) }] +------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +--------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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)) }], mode=[Sorted] +----------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }] +------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +--------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +----------------------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, @@ -2183,8 +2187,8 @@ Projection: sum1, sum2 ------------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]] --------------TableScan: aggregate_test_100 projection=[c1, c2, c9, c12] physical_plan -ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] ---GlobalLimitExec: skip=0, fetch=5 +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] ----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] ------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] --------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)) }], mode=[Sorted] @@ -2551,8 +2555,8 @@ Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, su ----------------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 ------------------TableScan: annotated_data_finite projection=[ts, inc_col, desc_col] physical_plan -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] ---GlobalLimitExec: skip=0, fetch=5 +GlobalLimitExec: skip=0, fetch=5 +--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] ----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] ------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] --------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)) }, 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)) }], mode=[Sorted] @@ -2713,8 +2717,8 @@ Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 ------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_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 --------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -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] ---GlobalLimitExec: skip=0, fetch=5 +GlobalLimitExec: skip=0, fetch=5 +--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] ----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] ------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] --------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)) }, 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)) }, 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)) }, 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)) }, 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)) }], mode=[Sorted] @@ -2767,8 +2771,8 @@ Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 ----------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]] ------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -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] ---GlobalLimitExec: skip=0, fetch=5 +GlobalLimitExec: skip=0, fetch=5 +--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] ----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] ------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] --------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)) }, 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)) }, 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)) }], mode=[Sorted] @@ -2960,16 +2964,17 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_da ----------------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 ------------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -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] +ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 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@7 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@8 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@13 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@14 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@3 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@4 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@9 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@10 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@5 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@6 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@11 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@12 as sum12] --GlobalLimitExec: skip=0, fetch=5 -----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)) }, 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)) }], mode=[Linear] -------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)) }, 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)) }], mode=[PartiallySorted([1, 0])] ---------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)) }, 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)) }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[PartiallySorted([0])] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[PartiallySorted([0, 1])] ---------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", 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)) }], mode=[Sorted] -----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +----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.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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] +------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)) }, 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)) }], mode=[Linear] +--------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)) }, 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)) }], mode=[PartiallySorted([1, 0])] +----------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)) }, 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)) }], mode=[Sorted] +------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[PartiallySorted([0])] +--------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[PartiallySorted([0, 1])] +----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", 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)) }], mode=[Sorted] +------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +--------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3362,8 +3367,8 @@ Projection: min1, max1 --------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]] ----------TableScan: aggregate_test_100 projection=[c3, c12] physical_plan -ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] ---GlobalLimitExec: skip=0, fetch=5 +GlobalLimitExec: skip=0, fetch=5 +--ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] ----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] ------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] --------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 }, 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 }], mode=[Sorted] @@ -4027,13 +4032,14 @@ Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregat --------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]] ----------TableScan: aggregate_test_100 projection=[c3, c4, c9] physical_plan -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] +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@2 as sum2, sum1@1 as sum1] --GlobalLimitExec: skip=0, fetch=5 -----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)) }] -------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] ---------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 }], mode=[Sorted] -----------SortExec: expr=[c3@0 + c4@1 DESC] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true +----ProjectionExec: expr=[c3@0 as c3, sum1@3 as sum1, 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 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] +------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)) }] +--------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] +----------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 }], mode=[Sorted] +------------SortExec: expr=[c3@0 + c4@1 DESC] +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true query III SELECT c3, From 23e495462bcd489d04d603c1fd15dcdf65dc6ecf Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 1 Mar 2024 16:31:10 +0300 Subject: [PATCH 25/85] fix clippy --- datafusion/physical-plan/src/projection.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 506e115b1e79..d05d61303e2d 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -39,7 +39,6 @@ use crate::{ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::equivalence::ProjectionMapping; From c9dce820c997d693e3a1abe63e601a30a154938a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 4 Mar 2024 12:56:03 +0300 Subject: [PATCH 26/85] ProjectionMapping refactor --- .../optimize_projections.rs | 3 +- .../physical-expr/src/equivalence/mod.rs | 2 +- .../src/equivalence/projection.rs | 47 +++++++++---------- .../src/equivalence/properties.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/projection.rs | 25 +++++----- 6 files changed, 39 insertions(+), 44 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 6f703d59a48a..f3b39491ac43 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -226,7 +226,7 @@ impl ProjectionOptimizer { 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)); } @@ -264,7 +264,6 @@ impl ProjectionOptimizer { else { return Ok(Transformed::No(self)); }; - // The projection must have all column expressions without aliases. let Some(projection_columns) = collect_alias_free_columns(projection_exec.expr()) else { diff --git a/datafusion/physical-expr/src/equivalence/mod.rs b/datafusion/physical-expr/src/equivalence/mod.rs index a31be06ecf0b..e7e730da259f 100644 --- a/datafusion/physical-expr/src/equivalence/mod.rs +++ b/datafusion/physical-expr/src/equivalence/mod.rs @@ -289,7 +289,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 fbd22cc5ac5a..e8669cfd6c6a 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -21,7 +21,6 @@ use crate::expressions::Column; use crate::PhysicalExpr; use arrow::datatypes::SchemaRef; -use arrow_schema::Schema; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::Result; @@ -49,15 +48,29 @@ 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 _; - Self::update_expr_with_input_schema(expression.clone(), input_schema) + let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; + expression + .transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => { + // Sometimes, an expression and its name in the input_schema + // doesn't match. This can cause problems, so we make sure + // that the expression name matches with the name in `input_schema`. + // Conceptually, `source_expr` and `expression` should be the same. + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + }) .map(|source_expr| (source_expr, target_expr)) }) .collect::>>() @@ -90,22 +103,6 @@ impl ProjectionMapping { .find(|(source, _)| source.eq(expr)) .map(|(_, target)| target.clone()) } - - /// This function ensures that an expression has the same name with the field of it in its input schema. - pub fn update_expr_with_input_schema( - expr: Arc, - input_schema: &Schema, - ) -> Result> { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => { - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - }) - } } #[cfg(test)] @@ -608,7 +605,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 @@ -691,7 +688,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)?; @@ -876,7 +873,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)?; @@ -1112,7 +1109,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 5a9a4f64876d..f2bd70772df2 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -1321,7 +1321,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 @@ -1331,7 +1331,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-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 922f9a89bfba..12b6e8786fad 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -357,7 +357,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/projection.rs b/datafusion/physical-plan/src/projection.rs index d05d61303e2d..9e9e4fd58fc8 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -69,17 +69,18 @@ impl ProjectionExec { input: Arc, ) -> Result { let input_schema = input.schema(); - expr = expr - .into_iter() - .map(|(expression, name)| { - ProjectionMapping::update_expr_with_input_schema( - expression, - &input_schema, - ) - .map(|source_expr| (source_expr, name.to_string())) - }) - .collect::>>()?; - + // construct a map from the input expressions to the output expression of the Projection + let expr_names = expr + .iter() + .map(|(_, alias)| alias.to_string()) + .collect::>(); + let projection_mapping = ProjectionMapping::try_new(expr, &input_schema)?; + expr = projection_mapping + .map + .iter() + .zip(expr_names) + .map(|((source, _), name)| (source.clone(), name)) + .collect(); let fields: Result> = expr .iter() .map(|(e, name)| { @@ -101,8 +102,6 @@ impl ProjectionExec { 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 cache = Self::compute_properties(&input, &projection_mapping, schema.clone())?; Ok(Self { From 5fe50fc101ff9bdf5d0f34c9342bc6d9c8d2aeb5 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 4 Mar 2024 14:44:27 +0300 Subject: [PATCH 27/85] Sync all joins --- .../optimize_projections.rs | 428 +++++++++++------- datafusion/physical-plan/src/projection.rs | 3 +- 2 files changed, 263 insertions(+), 168 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index f3b39491ac43..427ac046d5b5 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -302,40 +302,31 @@ impl ProjectionOptimizer { return Ok(Transformed::No(self)); }; - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); let (used_columns, unused_columns) = split_column_requirements(&requirement_map); if unused_columns.is_empty() { // All projected items are used. return Ok(Transformed::No(self)); } - let projected_exprs = projection_exec - .expr() - .iter() - .enumerate() - .filter_map(|(idx, (expr, alias))| { - if used_columns - .iter() - .map(|column| column.index()) - .collect::>() - .contains(&idx) - { - Some((expr.clone(), alias.clone())) - } else { - None - } - }) - .collect::>(); - let new_mapping = + let projected_exprs = + filter_unused_columns(projection_exec.expr(), &used_columns); + let mut new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); + let new_child_requirements = self + .required_columns + .iter() + .flat_map(|column| collect_columns(&projected_exprs[column.index()].0)) + .collect::>(); let new_projection_plan = Arc::new(ProjectionExec::try_new( - projected_exprs.clone(), + projected_exprs, self.children_nodes[0].plan.clone(), )?); + let new_projection_requires = self .required_columns - .iter() - .map(|col| new_mapping.get(col).cloned().unwrap_or(col.clone())) + .into_iter() + .map(|col| new_mapping.remove(&col).unwrap_or(col)) .collect(); let mut new_node = ProjectionOptimizer { plan: new_projection_plan, @@ -345,12 +336,7 @@ impl ProjectionOptimizer { }; // Since the rule work on the child node now, we need to insert child note requirements here. - new_node.children_nodes[0].required_columns = self - .required_columns - .iter() - .flat_map(|column| collect_columns(&projected_exprs[column.index()].0)) - .collect::>(); - + new_node.children_nodes[0].required_columns = new_child_requirements; Ok(Transformed::Yes(new_node)) } @@ -406,18 +392,19 @@ impl ProjectionOptimizer { let plan = self.plan.clone(); // These plans preserve the input schema, and do not add new requirements. - if let Some(coal_b) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_coalesce_batches(coal_b)?; + if let Some(coal_batches) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_coalesce_batches(coal_batches)?; } else if plan .as_any() .downcast_ref::() .is_some() { self = self.try_insert_below_coalesce_partitions()?; - } else if let Some(glimit) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_global_limit(glimit)?; - } else if let Some(llimit) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_local_limit(llimit)?; + } else if let Some(global_limit) = plan.as_any().downcast_ref::() + { + self = self.try_insert_below_global_limit(global_limit)?; + } else if let Some(local_limit) = plan.as_any().downcast_ref::() { + self = self.try_insert_below_local_limit(local_limit)?; } // These plans also preserve the input schema, but may extend requirements. else if let Some(filter) = plan.as_any().downcast_ref::() { @@ -427,10 +414,10 @@ impl ProjectionOptimizer { self = self.try_insert_below_repartition(repartition)?; } else if let Some(sort) = plan.as_any().downcast_ref::() { self = self.try_insert_below_sort(sort)?; - } else if let Some(sortp_merge) = + } else if let Some(sort_merge) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_sort_preserving_merge(sortp_merge)?; + self = self.try_insert_below_sort_preserving_merge(sort_merge)?; } // Preserves schema and do not change requirements, but have multi-child. else if plan.as_any().downcast_ref::().is_some() { @@ -513,7 +500,7 @@ impl ProjectionOptimizer { coal_batches: &CoalesceBatchesExec, ) -> Result { // CoalesceBatchesExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); @@ -544,7 +531,7 @@ impl ProjectionOptimizer { /// * `Result`: The potentially modified `ProjectionOptimizer`. fn try_insert_below_coalesce_partitions(mut self) -> Result { // CoalescePartitionsExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); @@ -577,7 +564,7 @@ impl ProjectionOptimizer { global_limit: &GlobalLimitExec, ) -> Result { // GlobalLimitExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { // if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = @@ -611,17 +598,19 @@ impl ProjectionOptimizer { /// * `Result`: The updated `ProjectionOptimizer`. fn try_insert_below_local_limit( mut self, - limit: &LocalLimitExec, + local_limit: &LocalLimitExec, ) -> Result { // LocalLimitExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = self.analyze_requirements(); + 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 (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - let plan = - Arc::new(LocalLimitExec::new(new_child.plan.clone(), limit.fetch())) as _; + let plan = Arc::new(LocalLimitExec::new( + new_child.plan.clone(), + local_limit.fetch(), + )) as _; self = ProjectionOptimizer { plan, @@ -651,7 +640,7 @@ impl ProjectionOptimizer { self.required_columns .extend(collect_columns(filter.predicate())); - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); @@ -694,7 +683,7 @@ impl ProjectionOptimizer { .extend(exprs.iter().flat_map(collect_columns)); } - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); @@ -738,7 +727,7 @@ impl ProjectionOptimizer { .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), ); - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); @@ -774,17 +763,17 @@ impl ProjectionOptimizer { /// * `Result`: The modified `ProjectionOptimizer`. fn try_insert_below_sort_preserving_merge( mut self, - sortp_merge: &SortPreservingMergeExec, + sort_merge: &SortPreservingMergeExec, ) -> Result { // SortPreservingMergeExec extends the requirements with the columns in its sort expressions. self.required_columns.extend( - sortp_merge + sort_merge .expr() .iter() .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), ); - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); @@ -792,10 +781,10 @@ impl ProjectionOptimizer { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; // Rewrite the sort expressions with possibly updated column indices. let new_sort_exprs = - update_sort_expressions(sortp_merge.expr(), &schema_mapping); + update_sort_expressions(sort_merge.expr(), &schema_mapping); let plan = Arc::new( SortPreservingMergeExec::new(new_sort_exprs, new_child.plan.clone()) - .with_fetch(sortp_merge.fetch()), + .with_fetch(sort_merge.fetch()), ) as _; self = ProjectionOptimizer { @@ -820,7 +809,7 @@ impl ProjectionOptimizer { /// * `Result`: The optimized `ProjectionOptimizer` after potentially inserting projection nodes. fn try_insert_below_union(mut self) -> Result { // UnionExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { let required_columns = mem::take(&mut self.required_columns); self.children_nodes @@ -853,7 +842,7 @@ impl ProjectionOptimizer { /// # Returns /// * `Result`: The potentially modified `ProjectionOptimizer` after the optimization process. fn try_insert_below_interleave(mut self) -> Result { - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { let required_columns = mem::take(&mut self.required_columns); self.children_nodes @@ -929,7 +918,7 @@ impl ProjectionOptimizer { (false, true) => { let required_columns = self.required_columns.clone(); let mut right_child = self.children_nodes.swap_remove(1); - let (new_left_child, left_schema_mapping) = + let (new_left_child, mut left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; right_child.required_columns = required_columns .iter() @@ -940,7 +929,22 @@ impl ProjectionOptimizer { 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( + 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::>(), + ); self = ProjectionOptimizer { plan, required_columns: HashSet::new(), @@ -1067,7 +1071,6 @@ impl ProjectionOptimizer { analyzed_join_left, analyzed_join_right, )?; - let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), new_right_child.plan.clone(), @@ -1139,8 +1142,8 @@ impl ProjectionOptimizer { } } (true, false) => { - let mut left_child = self.children_nodes.swap_remove(0); let required_columns = self.required_columns.clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( hj.on(), &HashMap::new(), @@ -1167,7 +1170,6 @@ impl ProjectionOptimizer { .filter(|col| col.index() < left_size) .cloned() .collect(); - let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1261,7 +1263,6 @@ impl ProjectionOptimizer { &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; @@ -1310,6 +1311,16 @@ impl ProjectionOptimizer { ) -> 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 = self + .required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() + } + _ => {} + } self.required_columns .extend(collect_columns_in_join_conditions( &[], @@ -1459,11 +1470,9 @@ impl ProjectionOptimizer { &analyzed_join_left, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; + 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(), @@ -1507,6 +1516,7 @@ impl ProjectionOptimizer { ); 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(), @@ -1549,6 +1559,16 @@ impl ProjectionOptimizer { ) -> 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 = self + .required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() + } + _ => {} + } self.required_columns .extend(collect_columns_in_join_conditions( smj.on(), @@ -1606,7 +1626,8 @@ impl ProjectionOptimizer { } } (false, true) => { - let right_child = self.children_nodes.swap_remove(1); + let required_columns = self.required_columns.clone(); + let mut right_child = self.children_nodes.swap_remove(1); let new_on = update_equivalence_conditions( smj.on(), &analyzed_join_left, @@ -1617,11 +1638,13 @@ impl ProjectionOptimizer { &analyzed_join_right, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; + 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(), @@ -1631,7 +1654,22 @@ impl ProjectionOptimizer { smj.sort_options.clone(), smj.null_equals_null, )?) as _; - + let new_left_size = new_left_child.plan.schema().fields().len(); + 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::>(), + ); self = ProjectionOptimizer { plan, required_columns: HashSet::new(), @@ -1640,7 +1678,8 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes.swap_remove(0); + let required_columns = self.required_columns.clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( smj.on(), &HashMap::new(), @@ -1651,8 +1690,22 @@ impl ProjectionOptimizer { &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, right_schema_mapping) = self + let (new_right_child, mut right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; + right_schema_mapping = right_schema_mapping + .into_iter() + .map(|(old, new)| { + ( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + left_size), + ) + }) + .collect(); + left_child.required_columns = required_columns + .iter() + .filter(|col| col.index() < left_size) + .cloned() + .collect(); let plan = Arc::new(SortMergeJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1693,11 +1746,9 @@ impl ProjectionOptimizer { &analyzed_join_left, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; + 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(), @@ -1749,6 +1800,7 @@ impl ProjectionOptimizer { ); 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(), @@ -1794,6 +1846,16 @@ impl ProjectionOptimizer { ) -> 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 = self + .required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() + } + _ => {} + } self.required_columns .extend(collect_columns_in_join_conditions( shj.on(), @@ -1855,7 +1917,8 @@ impl ProjectionOptimizer { } } (false, true) => { - let right_child = self.children_nodes.swap_remove(1); + let required_columns = self.required_columns.clone(); + let mut right_child = self.children_nodes.swap_remove(1); let new_on = update_equivalence_conditions( shj.on(), &analyzed_join_left, @@ -1866,11 +1929,13 @@ impl ProjectionOptimizer { &analyzed_join_right, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; + 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(SymmetricHashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -1882,7 +1947,22 @@ impl ProjectionOptimizer { shj.right_sort_exprs().map(|exprs| exprs.to_vec()), shj.partition_mode(), )?) as _; - + let new_left_size = new_left_child.plan.schema().fields().len(); + 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::>(), + ); self = ProjectionOptimizer { plan, required_columns: HashSet::new(), @@ -1891,7 +1971,8 @@ impl ProjectionOptimizer { } } (true, false) => { - let left_child = self.children_nodes.swap_remove(0); + let required_columns = self.required_columns.clone(); + let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( shj.on(), &HashMap::new(), @@ -1902,8 +1983,22 @@ impl ProjectionOptimizer { &HashMap::new(), &analyzed_join_right, ); - let (new_right_child, right_schema_mapping) = self + let (new_right_child, mut right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; + right_schema_mapping = right_schema_mapping + .into_iter() + .map(|(old, new)| { + ( + Column::new(old.name(), old.index() + left_size), + Column::new(new.name(), new.index() + left_size), + ) + }) + .collect(); + left_child.required_columns = required_columns + .iter() + .filter(|col| col.index() < left_size) + .cloned() + .collect(); let plan = Arc::new(SymmetricHashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1946,11 +2041,9 @@ impl ProjectionOptimizer { &analyzed_join_left, &HashMap::new(), ); - let (new_left_child, left_schema_mapping) = self - .insert_projection_below_single_child( - analyzed_join_left, - 0, - )?; + 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(), @@ -2004,6 +2097,7 @@ impl ProjectionOptimizer { ); 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(), @@ -2185,7 +2279,7 @@ impl ProjectionOptimizer { })); self.required_columns .extend(w_agg.partition_keys.iter().flat_map(collect_columns)); - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if !all_columns_required(&requirement_map) { if window_agg_required( w_agg.input().schema().fields().len(), @@ -2292,7 +2386,7 @@ impl ProjectionOptimizer { })); self.required_columns .extend(bw_agg.partition_keys.iter().flat_map(collect_columns)); - let requirement_map = self.analyze_requirements(); + let requirement_map = analyze_requirements(&self); if !all_columns_required(&requirement_map) { if window_agg_required( bw_agg.input().schema().fields().len(), @@ -2384,18 +2478,6 @@ impl ProjectionOptimizer { Ok(self) } - /// Compares the required and existing columns in the node, and maps them accordingly. Caller side must - /// ensure that the node extends its own requirements if the node's plan can introduce new requirements. - fn analyze_requirements(&self) -> ColumnRequirements { - let mut requirement_map = HashMap::new(); - let columns_in_schema = collect_columns_in_plan_schema(&self.plan); - columns_in_schema.into_iter().for_each(|col| { - let contains = self.required_columns.contains(&col); - requirement_map.insert(col, contains); - }); - requirement_map - } - /// 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 @@ -2472,32 +2554,6 @@ impl ProjectionOptimizer { Ok((inserted_projection_nodes, new_mapping)) } - /// Single child version of `insert_projection` for joins. - fn insert_projection_below_single_child( - mut self, - requirement_map_left: ColumnRequirements, - children_index: usize, - ) -> Result<(Self, HashMap)> { - // During the iteration, we construct the ProjectionExec with required columns as the new child, - // and also collect the unused columns to store the index changes after removal of some columns. - let (used_columns, unused_columns) = - split_column_requirements(&requirement_map_left); - let child_plan = self.plan.children().remove(children_index); - 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: HashMap::new(), - children_nodes: vec![self.children_nodes.swap_remove(children_index)], - }; - Ok((inserted_projection, new_mapping)) - } - /// Single child version of `insert_projection` for joins. #[allow(clippy::type_complexity)] fn insert_projection_below_multi_child( @@ -2741,7 +2797,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(_coal_parts) = plan_any.downcast_ref::() { @@ -2751,7 +2807,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + 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 @@ -2759,7 +2815,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + 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 @@ -2767,7 +2823,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + 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 @@ -2775,7 +2831,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + 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 @@ -2783,7 +2839,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(_cj) = plan_any.downcast_ref::() { self.plan = self.plan.with_new_children( self.children_nodes @@ -2791,7 +2847,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(projection) = plan_any.downcast_ref::() { self.plan = rewrite_projection( projection, @@ -2806,21 +2862,21 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.clone(), &all_mappings[0], )?; - self.update_mapping(all_mappings) + 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], )?; - self.update_mapping(all_mappings) + 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], )?; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(sortp_merge) = plan_any.downcast_ref::() { @@ -2829,7 +2885,7 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.clone(), &all_mappings[0], )?; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(hj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); let left_mapping = all_mappings.swap_remove(0); @@ -2872,7 +2928,7 @@ impl ProjectionOptimizer { } }; // self.schema_mapping = all_mappings; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(nlj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); let left_mapping = all_mappings.swap_remove(0); @@ -2914,7 +2970,7 @@ impl ProjectionOptimizer { all_mappings.push(right_mapping) } }; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(smj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); let left_mapping = all_mappings.swap_remove(0); @@ -2945,7 +3001,7 @@ impl ProjectionOptimizer { JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, JoinType::RightAnti | JoinType::RightSemi => right_mapping, }; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(shj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); let left_mapping = all_mappings.swap_remove(0); @@ -2976,7 +3032,7 @@ impl ProjectionOptimizer { JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, JoinType::RightAnti | JoinType::RightSemi => right_mapping, }; - self.update_mapping(all_mappings) + update_mapping(&mut self, all_mappings) } else if let Some(agg) = plan_any.downcast_ref::() { if agg.aggr_expr().iter().any(|expr| { expr.clone() @@ -3006,7 +3062,7 @@ impl ProjectionOptimizer { } else { return Ok(Transformed::No(self)); }; - self.update_mapping(all_mappings) + 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, @@ -3017,7 +3073,7 @@ impl ProjectionOptimizer { } else { return Ok(Transformed::No(self)); }; - self.update_mapping(all_mappings) + 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::>(); @@ -3071,24 +3127,6 @@ impl ProjectionOptimizer { Ok(Transformed::Yes(self)) } - fn update_mapping(&mut self, mut child_mappings: Vec>) { - if self.schema_mapping.is_empty() { - self.schema_mapping = child_mappings.swap_remove(0); - } else { - let child_map = child_mappings.swap_remove(0); - self.schema_mapping = self - .schema_mapping - .iter() - .map(|(initial, new)| { - ( - initial.clone(), - child_map.get(new).cloned().unwrap_or(new.clone()), - ) - }) - .collect() - } - } - /// After the top-down pass, there may be some unnecessary projections surviving /// since they assumes themselves as necessary when they are analyzed, but after /// some optimizations below, they may become unnecessary. This function checks @@ -3344,6 +3382,18 @@ fn satisfy_initial_schema( } } +/// 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 = HashMap::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 +} + fn preserve_requirements(po: ProjectionOptimizer) -> Result { if po.schema_mapping.is_empty() { // The initial schema is already satisfied, no further action required. @@ -3463,6 +3513,29 @@ fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec, String)], + used_columns: &HashSet, +) -> Vec<(Arc, String)> { + projection_exprs + .iter() + .enumerate() + .filter_map(|(idx, (expr, alias))| { + if used_columns + .iter() + .map(|column| column.index()) + .collect::>() + .contains(&idx) + { + Some((expr.clone(), alias.clone())) + } else { + None + } + }) + .collect::>() +} + /// 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 @@ -3798,6 +3871,27 @@ fn update_non_equivalence_conditions( }) } +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() + } +} + /// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. /// /// # Arguments diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 9e9e4fd58fc8..2dda1018e62c 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -69,11 +69,12 @@ impl ProjectionExec { input: Arc, ) -> Result { let input_schema = input.schema(); - // construct a map from the input expressions to the output expression of the Projection let expr_names = expr .iter() .map(|(_, alias)| alias.to_string()) .collect::>(); + + // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(expr, &input_schema)?; expr = projection_mapping .map From 6ed6878229340111620c06ca2de73747141207b4 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 4 Mar 2024 15:37:50 +0300 Subject: [PATCH 28/85] Adapt test changes --- datafusion/sqllogictest/test_files/group_by.slt | 10 +++++----- datafusion/sqllogictest/test_files/limit.slt | 2 +- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 5cc5a3fae544..45243e300c2b 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2882,20 +2882,20 @@ SortExec: expr=[sn@2 ASC NULLS LAST] --------------MemoryExec: partitions=1, partition_sizes=[1] --------------MemoryExec: partitions=1, partition_sizes=[1] -query ITIPTR rowsort +query ITIPTR SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate FROM sales_global AS s JOIN sales_global AS e ON s.currency = e.currency AND s.ts >= e.ts GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency -ORDER BY s.sn +ORDER BY s.sn, s.zip_code ---- 0 GRC 0 2022-01-01T06:00:00 EUR 30 -0 GRC 4 2022-01-03T10:00:00 EUR 80 1 FRA 1 2022-01-01T08:00:00 EUR 50 -1 FRA 3 2022-01-02T12:00:00 EUR 200 1 TUR 2 2022-01-01T11:30:00 TRY 75 +1 FRA 3 2022-01-02T12:00:00 EUR 200 +0 GRC 4 2022-01-03T10:00:00 EUR 80 1 TUR 4 2022-01-03T10:00:00 TRY 100 # Run order-sensitive aggregators in multiple partitions @@ -3215,7 +3215,7 @@ JOIN sales_global AS e ON s.currency = e.currency AND s.ts >= e.ts GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency -ORDER BY s.sn +ORDER BY s.sn, s.zip_code ---- 0 GRC 0 2022-01-01T06:00:00 EUR 30 1 FRA 1 2022-01-01T08:00:00 EUR 50 diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 3748253be046..d891396c4477 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -400,7 +400,7 @@ AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] --CoalesceBatchesExec: target_batch_size=8192 ----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 ------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] ---------MemoryExec: partitions=4, partition_sizes=[1, 1, 2, 1] +--------MemoryExec: partitions=4, partition_sizes=[1, 2, 1, 1] query I SELECT i FROM t1000 ORDER BY i DESC LIMIT 3; From f757233c4a154a46b4408add99374776821c6372 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 4 Mar 2024 18:15:56 +0300 Subject: [PATCH 29/85] Remove duplications --- .../optimize_projections.rs | 428 +++++++++--------- datafusion/physical-plan/src/projection.rs | 1 + 2 files changed, 222 insertions(+), 207 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 427ac046d5b5..56820b24a2b6 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -182,7 +182,8 @@ impl ProjectionOptimizer { // The projection can get narrower. self = match self.try_narrow_projection()? { Transformed::Yes(narrowed) => { - return Ok(narrowed); + // We need to re-run the rule on the new node since it may be removed within the source provider. + return narrowed.optimize_projections(); } Transformed::No(no_change) => no_change, }; @@ -313,11 +314,6 @@ impl ProjectionOptimizer { let mut new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); - let new_child_requirements = self - .required_columns - .iter() - .flat_map(|column| collect_columns(&projected_exprs[column.index()].0)) - .collect::>(); let new_projection_plan = Arc::new(ProjectionExec::try_new( projected_exprs, self.children_nodes[0].plan.clone(), @@ -328,16 +324,13 @@ impl ProjectionOptimizer { .into_iter() .map(|col| new_mapping.remove(&col).unwrap_or(col)) .collect(); - let mut new_node = ProjectionOptimizer { + + Ok(Transformed::Yes(ProjectionOptimizer { plan: new_projection_plan, required_columns: new_projection_requires, schema_mapping: new_mapping, children_nodes: self.children_nodes, - }; - - // Since the rule work on the child node now, we need to insert child note requirements here. - new_node.children_nodes[0].required_columns = new_child_requirements; - Ok(Transformed::Yes(new_node)) + })) } /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. @@ -918,32 +911,23 @@ impl ProjectionOptimizer { (false, true) => { let required_columns = self.required_columns.clone(); let mut right_child = self.children_nodes.swap_remove(1); + 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(); + + right_child.required_columns = + update_right_child_requirements(&required_columns, left_size); + 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( - 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 = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, ); self = ProjectionOptimizer { plan, @@ -958,20 +942,13 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let (new_right_child, mut right_schema_mapping) = self.insert_projection_below_right_child(analyzed_join_right)?; - right_schema_mapping = right_schema_mapping - .into_iter() - .map(|(old, new)| { - ( - Column::new(old.name(), old.index() + left_size), - Column::new(new.name(), new.index() + left_size), - ) - }) - .collect(); - left_child.required_columns = required_columns - .iter() - .filter(|col| col.index() < left_size) - .cloned() - .collect(); + + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + filter_left_requirements(required_columns, left_size); + let plan = Arc::new(CrossJoinExec::new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1024,11 +1001,8 @@ impl ProjectionOptimizer { // HashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. match hj.join_type() { JoinType::RightAnti | JoinType::RightSemi => { - self.required_columns = self - .required_columns - .into_iter() - .map(|col| Column::new(col.name(), col.index() + left_size)) - .collect() + self.required_columns = + update_right_requirements(self.required_columns, left_size); } _ => {} } @@ -1040,6 +1014,7 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.schema(), self.children_nodes[1].plan.schema(), )); + let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( hj.left(), hj.right(), @@ -1091,6 +1066,7 @@ impl ProjectionOptimizer { (false, true) => { let required_columns = self.required_columns.clone(); let mut right_child = self.children_nodes.swap_remove(1); + let new_on = update_equivalence_conditions( hj.on(), &analyzed_join_left, @@ -1101,13 +1077,13 @@ impl ProjectionOptimizer { &analyzed_join_right, &HashMap::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(); + + right_child.required_columns = + update_right_child_requirements(&required_columns, left_size); + let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -1118,22 +1094,12 @@ impl ProjectionOptimizer { hj.null_equals_null(), )?) as _; let new_left_size = new_left_child.plan.schema().fields().len(); - 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 = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, ); - self = ProjectionOptimizer { plan, required_columns: HashSet::new(), @@ -1156,20 +1122,13 @@ impl ProjectionOptimizer { ); let (new_right_child, mut right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; - right_schema_mapping = right_schema_mapping - .into_iter() - .map(|(old, new)| { - ( - Column::new(old.name(), old.index() + left_size), - Column::new(new.name(), new.index() + left_size), - ) - }) - .collect(); - left_child.required_columns = required_columns - .iter() - .filter(|col| col.index() < left_size) - .cloned() - .collect(); + + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + filter_left_requirements(required_columns, left_size); + let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1313,11 +1272,8 @@ impl ProjectionOptimizer { // 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 = self - .required_columns - .into_iter() - .map(|col| Column::new(col.name(), col.index() + left_size)) - .collect() + self.required_columns = + update_right_requirements(self.required_columns, left_size); } _ => {} } @@ -1379,11 +1335,10 @@ impl ProjectionOptimizer { ); 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(); + + 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(), @@ -1391,20 +1346,11 @@ impl ProjectionOptimizer { nlj.join_type(), )?) as _; let new_left_size = new_left_child.plan.schema().fields().len(); - 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 = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, ); self = ProjectionOptimizer { @@ -1424,20 +1370,13 @@ impl ProjectionOptimizer { ); let (new_right_child, mut right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; - right_schema_mapping = right_schema_mapping - .into_iter() - .map(|(old, new)| { - ( - Column::new(old.name(), old.index() + left_size), - Column::new(new.name(), new.index() + left_size), - ) - }) - .collect(); - left_child.required_columns = required_columns - .iter() - .filter(|col| col.index() < left_size) - .cloned() - .collect(); + + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + filter_left_requirements(required_columns, left_size); + let plan = Arc::new(NestedLoopJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1561,11 +1500,8 @@ impl ProjectionOptimizer { // 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 = self - .required_columns - .into_iter() - .map(|col| Column::new(col.name(), col.index() + left_size)) - .collect() + self.required_columns = + update_right_requirements(self.required_columns, left_size); } _ => {} } @@ -1655,20 +1591,11 @@ impl ProjectionOptimizer { smj.null_equals_null, )?) as _; let new_left_size = new_left_child.plan.schema().fields().len(); - 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 = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, ); self = ProjectionOptimizer { plan, @@ -1692,20 +1619,12 @@ impl ProjectionOptimizer { ); let (new_right_child, mut right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; - right_schema_mapping = right_schema_mapping - .into_iter() - .map(|(old, new)| { - ( - Column::new(old.name(), old.index() + left_size), - Column::new(new.name(), new.index() + left_size), - ) - }) - .collect(); - left_child.required_columns = required_columns - .iter() - .filter(|col| col.index() < left_size) - .cloned() - .collect(); + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + filter_left_requirements(required_columns, left_size); + let plan = Arc::new(SortMergeJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1848,11 +1767,8 @@ impl ProjectionOptimizer { // 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 = self - .required_columns - .into_iter() - .map(|col| Column::new(col.name(), col.index() + left_size)) - .collect() + self.required_columns = + update_right_requirements(self.required_columns, left_size); } _ => {} } @@ -1895,6 +1811,14 @@ impl ProjectionOptimizer { 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(), @@ -1903,9 +1827,8 @@ impl ProjectionOptimizer { new_filter, shj.join_type(), shj.null_equals_null(), - // TODO: update these - shj.left_sort_exprs().map(|exprs| exprs.to_vec()), - shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + new_left_sort_exprs, + new_right_sort_exprs, shj.partition_mode(), )?) as _; @@ -1931,11 +1854,10 @@ impl ProjectionOptimizer { ); 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(); + + 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(), @@ -1948,20 +1870,11 @@ impl ProjectionOptimizer { shj.partition_mode(), )?) as _; let new_left_size = new_left_child.plan.schema().fields().len(); - 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 = extend_left_mapping_with_right( + left_schema_mapping, + &right_child.plan, + left_size, + new_left_size, ); self = ProjectionOptimizer { plan, @@ -1985,20 +1898,12 @@ impl ProjectionOptimizer { ); let (new_right_child, mut right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; - right_schema_mapping = right_schema_mapping - .into_iter() - .map(|(old, new)| { - ( - Column::new(old.name(), old.index() + left_size), - Column::new(new.name(), new.index() + left_size), - ) - }) - .collect(); - left_child.required_columns = required_columns - .iter() - .filter(|col| col.index() < left_size) - .cloned() - .collect(); + + right_schema_mapping = + update_right_mapping(right_schema_mapping, left_size); + + left_child.required_columns = + filter_left_requirements(required_columns, left_size); let plan = Arc::new(SymmetricHashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -2993,13 +2898,24 @@ impl ProjectionOptimizer { &right_mapping, left_size, )?; - all_mappings[0] = match smj.join_type() { + match smj.join_type() { JoinType::Right | JoinType::Full | JoinType::Left - | JoinType::Inner => new_mapping, - JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, - JoinType::RightAnti | JoinType::RightSemi => right_mapping, + | 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) } else if let Some(shj) = plan_any.downcast_ref::() { @@ -3024,13 +2940,24 @@ impl ProjectionOptimizer { &right_mapping, left_size, )?; - all_mappings[0] = match shj.join_type() { + match shj.join_type() { JoinType::Right | JoinType::Full | JoinType::Left - | JoinType::Inner => new_mapping, - JoinType::LeftSemi | JoinType::LeftAnti => left_mapping, - JoinType::RightAnti | JoinType::RightSemi => right_mapping, + | 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) } else if let Some(agg) = plan_any.downcast_ref::() { @@ -3536,6 +3463,16 @@ fn filter_unused_columns( .collect::>() } +fn filter_left_requirements( + required_columns: HashSet, + left_size: usize, +) -> HashSet { + required_columns + .into_iter() + .filter(|col| col.index() < left_size) + .collect() +} + /// 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 @@ -3871,6 +3808,17 @@ fn update_non_equivalence_conditions( }) } +fn update_right_child_requirements( + required_columns: &HashSet, + left_size: usize, +) -> HashSet { + 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>, @@ -3892,6 +3840,54 @@ fn update_mapping( } } +fn update_right_mapping( + right_schema_mapping: HashMap, + left_size: usize, +) -> HashMap { + 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: HashSet, + left_size: usize, +) -> HashSet { + required_columns + .into_iter() + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect() +} + +fn extend_left_mapping_with_right( + mut left_schema_mapping: HashMap, + right_child_plan: &Arc, + left_size: usize, + new_left_size: usize, +) -> HashMap { + 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 +} + /// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. /// /// # Arguments @@ -4428,6 +4424,25 @@ fn rewrite_symmetric_hash_join( filter.schema().clone(), ) }); + + let new_left_sort_exprs = shj.left_sort_exprs().map(|exprs| { + exprs + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, left_mapping), + options: sort_expr.options, + }) + .collect() + }); + let new_right_sort_exprs = shj.left_sort_exprs().map(|exprs| { + exprs + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, right_mapping), + options: sort_expr.options, + }) + .collect() + }); SymmetricHashJoinExec::try_new( left_input_plan, right_input_plan, @@ -4435,9 +4450,8 @@ fn rewrite_symmetric_hash_join( new_filter, shj.join_type(), shj.null_equals_null(), - // TODO: update these - shj.left_sort_exprs().map(|exprs| exprs.to_vec()), - shj.right_sort_exprs().map(|exprs| exprs.to_vec()), + new_left_sort_exprs, + new_right_sort_exprs, shj.partition_mode(), ) .map(|plan| Arc::new(plan) as _) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 2dda1018e62c..b8a603a71e50 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -82,6 +82,7 @@ impl ProjectionExec { .zip(expr_names) .map(|((source, _), name)| (source.clone(), name)) .collect(); + let fields: Result> = expr .iter() .map(|(e, name)| { From 12c43070b10589317e92ff06044b1fe06d466ed9 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 5 Mar 2024 13:32:57 +0300 Subject: [PATCH 30/85] Update optimize_projections.rs --- .../optimize_projections.rs | 876 +++++++++--------- 1 file changed, 430 insertions(+), 446 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 56820b24a2b6..ae2a074ea66c 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -63,7 +63,9 @@ use crate::physical_plan::ExecutionPlan; use arrow_schema::SchemaRef; use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{ + ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, +}; use datafusion_common::{JoinSide, JoinType}; use datafusion_physical_expr::expressions::{Column, Literal}; use datafusion_physical_expr::utils::collect_columns; @@ -159,40 +161,40 @@ impl ProjectionOptimizer { // We first need to check having 2 sequential projections in case of merging them. if projection_input.as_any().is::() { self = match self.try_unifying_projections()? { - Transformed::Yes(unified_plans) => { + 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.optimize_projections(); + return unified_plans.data.optimize_projections(); } - Transformed::No(no_change) => no_change, + 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()? { - Transformed::Yes(removed) => { + 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.adjust_node_with_requirements(); + return removed.data.adjust_node_with_requirements(); } - Transformed::No(no_change) => no_change, + no_change => no_change.data, }; // The projection can get narrower. self = match self.try_narrow_projection()? { - Transformed::Yes(narrowed) => { + narrowed if narrowed.transformed => { // We need to re-run the rule on the new node since it may be removed within the source provider. - return narrowed.optimize_projections(); + return narrowed.data.optimize_projections(); } - Transformed::No(no_change) => no_change, + no_change => no_change.data, }; // Source providers: if projection_input.as_any().is::() { self = match self.try_projected_csv() { - Transformed::Yes(new_csv) => return Ok(new_csv), - Transformed::No(no_change) => no_change, + new_csv if new_csv.transformed => return Ok(new_csv.data), + no_change => no_change.data, } } // TODO: Other source execs can be implemented here if projection can be applied within the scope of them. @@ -218,18 +220,18 @@ impl ProjectionOptimizer { fn try_unifying_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)); + return Ok(Transformed::no(self)); }; let Some(child_projection) = self.children_nodes[0] .plan .as_any() .downcast_ref::() else { - return Ok(Transformed::No(self)); + 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)); + return Ok(Transformed::no(self)); } let mut projected_exprs = vec![]; @@ -237,7 +239,7 @@ impl ProjectionOptimizer { let Some(expr) = update_expr_with_projection(expr, child_projection.expr(), true)? else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; projected_exprs.push((expr, alias.clone())); } @@ -245,7 +247,7 @@ impl ProjectionOptimizer { let new_plan = ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) .map(|e| Arc::new(e) as _)?; - Ok(Transformed::Yes(Self { + Ok(Transformed::yes(Self { plan: new_plan, // Schema of the projection does not change, // so no need any update on state variables. @@ -263,12 +265,12 @@ impl ProjectionOptimizer { // It must be a projection let Some(projection_exec) = self.plan.as_any().downcast_ref::() else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; // The projection must have all column expressions without aliases. let Some(projection_columns) = collect_alias_free_columns(projection_exec.expr()) else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; // Input requirements of the projection in terms of projection's parent requirements: @@ -283,14 +285,14 @@ impl ProjectionOptimizer { &projection_columns, ); let new_current_node = self.children_nodes.swap_remove(0); - Ok(Transformed::Yes(ProjectionOptimizer { + 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)) + Ok(Transformed::no(self)) } } @@ -300,17 +302,16 @@ impl ProjectionOptimizer { // It must be a projection. let Some(projection_exec) = self.plan.as_any().downcast_ref::() else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; let requirement_map = analyze_requirements(&self); let (used_columns, unused_columns) = split_column_requirements(&requirement_map); if unused_columns.is_empty() { // All projected items are used. - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); } - let projected_exprs = - filter_unused_columns(projection_exec.expr(), &used_columns); + let projected_exprs = collect_used_columns(projection_exec.expr(), &used_columns); let mut new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); @@ -325,7 +326,7 @@ impl ProjectionOptimizer { .map(|col| new_mapping.remove(&col).unwrap_or(col)) .collect(); - Ok(Transformed::Yes(ProjectionOptimizer { + Ok(Transformed::yes(ProjectionOptimizer { plan: new_projection_plan, required_columns: new_projection_requires, schema_mapping: new_mapping, @@ -337,16 +338,16 @@ impl ProjectionOptimizer { fn try_projected_csv(self) -> Transformed { // These plans are known. let Some(projection) = self.plan.as_any().downcast_ref::() else { - return Transformed::No(self); + return Transformed::no(self); }; let Some(csv) = projection.input().as_any().downcast_ref::() else { - return Transformed::No(self); + return Transformed::no(self); }; // 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. let Some(projection_columns) = collect_alias_free_columns(projection.expr()) else { - return Transformed::No(self); + return Transformed::no(self); }; let mut file_scan = csv.base_config().clone(); @@ -360,7 +361,7 @@ impl ProjectionOptimizer { file_scan.projection = Some(new_projections); - Transformed::Yes(ProjectionOptimizer { + Transformed::yes(ProjectionOptimizer { plan: Arc::new(CsvExec::new( file_scan, csv.has_header(), @@ -947,7 +948,7 @@ impl ProjectionOptimizer { update_right_mapping(right_schema_mapping, left_size); left_child.required_columns = - filter_left_requirements(required_columns, left_size); + collect_left_used_columns(required_columns, left_size); let plan = Arc::new(CrossJoinExec::new( left_child.plan.clone(), @@ -1127,7 +1128,7 @@ impl ProjectionOptimizer { update_right_mapping(right_schema_mapping, left_size); left_child.required_columns = - filter_left_requirements(required_columns, left_size); + collect_left_used_columns(required_columns, left_size); let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), @@ -1375,7 +1376,7 @@ impl ProjectionOptimizer { update_right_mapping(right_schema_mapping, left_size); left_child.required_columns = - filter_left_requirements(required_columns, left_size); + collect_left_used_columns(required_columns, left_size); let plan = Arc::new(NestedLoopJoinExec::try_new( left_child.plan.clone(), @@ -1623,7 +1624,7 @@ impl ProjectionOptimizer { update_right_mapping(right_schema_mapping, left_size); left_child.required_columns = - filter_left_requirements(required_columns, left_size); + collect_left_used_columns(required_columns, left_size); let plan = Arc::new(SortMergeJoinExec::try_new( left_child.plan.clone(), @@ -1903,7 +1904,7 @@ impl ProjectionOptimizer { update_right_mapping(right_schema_mapping, left_size); left_child.required_columns = - filter_left_requirements(required_columns, left_size); + collect_left_used_columns(required_columns, left_size); let plan = Arc::new(SymmetricHashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -2811,7 +2812,6 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.clone(), &left_mapping, &right_mapping, - left_size, )?; match hj.join_type() { JoinType::Right @@ -2968,7 +2968,7 @@ impl ProjectionOptimizer { && !self.children_nodes[0].schema_mapping.is_empty() }) { self = preserve_requirements(self)?; - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); } self.plan = if let Some(updated) = rewrite_aggregate( agg, @@ -2977,7 +2977,7 @@ impl ProjectionOptimizer { )? { updated } else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; } else if let Some(w_agg) = plan_any.downcast_ref::() { self.plan = if let Some(updated) = rewrite_window_aggregate( @@ -2987,7 +2987,7 @@ impl ProjectionOptimizer { )? { updated } else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; update_mapping(&mut self, all_mappings) } else if let Some(bw_agg) = plan_any.downcast_ref::() { @@ -2998,7 +2998,7 @@ impl ProjectionOptimizer { )? { updated } else { - return Ok(Transformed::No(self)); + return Ok(Transformed::no(self)); }; update_mapping(&mut self, all_mappings) } else if let Some(_file_sink) = plan_any.downcast_ref::() { @@ -3051,7 +3051,7 @@ impl ProjectionOptimizer { )?; } - Ok(Transformed::Yes(self)) + Ok(Transformed::yes(self)) } /// After the top-down pass, there may be some unnecessary projections surviving @@ -3121,108 +3121,32 @@ impl ProjectionOptimizer { } } -impl TreeNode for ProjectionOptimizer { - fn apply_children(&self, op: &mut F) -> Result - where - F: FnMut(&Self) -> Result, - { - for child in &self.children_nodes { - match op(child)? { - VisitRecursion::Continue => {} - VisitRecursion::Skip => return Ok(VisitRecursion::Continue), - VisitRecursion::Stop => return Ok(VisitRecursion::Stop), - } - } - Ok(VisitRecursion::Continue) +impl ConcreteTreeNode for ProjectionOptimizer { + fn children(&self) -> Vec<&Self> { + self.children_nodes.iter().collect_vec() } - fn map_children(mut self, transform: F) -> Result - where - F: FnMut(Self) -> Result, - { - if self.children_nodes.is_empty() { - Ok(self) - } else { - self.children_nodes = self - .children_nodes - .into_iter() - .map(transform) - .collect::>>()?; - - self = match self.index_updater()? { - Transformed::Yes(updated) => updated, - Transformed::No(not_rewritable) => { - ProjectionOptimizer::new_default(not_rewritable.plan) - } - }; - - // After the top-down pass, there may be some unnecessary projections surviving - // since they assumes themselves as necessary when they are analyzed, but after - // some optimizations below, they may become unnecessary. This check is done - // here, and if the projection is regarded as unnecessary, the removal would - // set a new the mapping on the new node, which is the child of the projection. - self = self.try_remove_projection_bottom_up()?; - - Ok(self) - } + fn take_children(mut self) -> (Self, Vec) { + let children = mem::take(&mut self.children_nodes); + (self, children) } -} - -/// 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. -/// Each column in the requirement maps corresponds to its own table schema index, not to the join output schema. -/// -/// # 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: &HashSet, - 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); + 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), + }; - requirement_map_right = requirement_map_right - .into_iter() - .map(|(col, used)| (Column::new(col.name(), col.index() - left_size), used)) - .collect::>(); + // After the top-down pass, there may be some unnecessary projections surviving + // since they assumes themselves as necessary when they are analyzed, but after + // some optimizations below, they may become unnecessary. This 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()?; - (requirement_map_left, requirement_map_right) + Ok(self) + } } #[derive(Default)] @@ -3248,15 +3172,16 @@ impl PhysicalOptimizerRule for OptimizeProjections { // Insert the initial requirements to the root node, and run the rule. optimizer.required_columns = initial_requirements.clone(); let mut optimized = optimizer.transform_down(&|o| { - o.adjust_node_with_requirements().map(Transformed::Yes) + 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 = satisfy_initial_schema(optimized, initial_requirements)?; + optimized = optimized + .map_data(|node| satisfy_initial_schema(node, initial_requirements))?; - Ok(optimized.plan) + Ok(optimized.data.plan) } fn name(&self) -> &str { @@ -3268,6 +3193,32 @@ impl PhysicalOptimizerRule for OptimizeProjections { } } +// 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: HashMap = HashMap::new(); + // Collect the column references' usage in the parent projection. + projection.expr().iter().try_for_each(|(expr, _)| { + expr.apply(&mut |expr| { + Ok({ + if let Some(column) = expr.as_any().downcast_ref::() { + *column_ref_map.entry(column.clone()).or_default() += 1; + } + 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. @@ -3321,6 +3272,63 @@ fn analyze_requirements(node: &ProjectionOptimizer) -> ColumnRequirements { 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. +/// Each column in the requirement maps corresponds to its own table schema index, not to the join output schema. +/// +/// # 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: &HashSet, + 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) +} + fn preserve_requirements(po: ProjectionOptimizer) -> Result { if po.schema_mapping.is_empty() { // The initial schema is already satisfied, no further action required. @@ -3373,6 +3381,24 @@ 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. +/// +/// # Arguments +/// * `input_columns`: Reference to a `HashSet` representing the input columns. +/// * `projection_requires`: Reference to a `HashSet` representing the projection requirements. +/// +/// # Returns +/// `true` if all input columns are required, otherwise `false`. +/// +fn all_input_columns_required( + input_columns: &HashSet, + projection_requires: &HashSet, +) -> bool { + input_columns + .iter() + .all(|input_column| projection_requires.contains(input_column)) +} + fn window_agg_required( original_schema_len: usize, requirements: &ColumnRequirements, @@ -3383,32 +3409,6 @@ fn window_agg_required( .any(|(_column, used)| *used) } -// If an expression is not trivial and it is referred more than 1, -// unification will not be beneficial as going against caching mechanism -// for non-trivial computations. See the discussion: -// https://github.com/apache/arrow-datafusion/issues/8296 -fn caching_projections( - projection: &ProjectionExec, - child_projection: &ProjectionExec, -) -> Result { - let mut column_ref_map: HashMap = HashMap::new(); - // Collect the column references' usage in the parent projection. - projection.expr().iter().try_for_each(|(expr, _)| { - expr.apply(&mut |expr| { - Ok({ - if let Some(column) = expr.as_any().downcast_ref::() { - *column_ref_map.entry(column.clone()).or_default() += 1; - } - VisitRecursion::Continue - }) - })?; - Ok(()) as Result<()> - })?; - Ok(column_ref_map.iter().any(|(column, count)| { - *count > 1 && !is_expr_trivial(&child_projection.expr()[column.index()].0) - })) -} - /// Checks if the given expression is trivial. /// An expression is considered trivial if it is either a `Column` or a `Literal`. fn is_expr_trivial(expr: &Arc) -> bool { @@ -3416,23 +3416,6 @@ fn is_expr_trivial(expr: &Arc) -> bool { || expr.as_any().downcast_ref::().is_some() } -/// 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 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) -} - /// 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. @@ -3440,39 +3423,6 @@ fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec, String)], - used_columns: &HashSet, -) -> Vec<(Arc, String)> { - projection_exprs - .iter() - .enumerate() - .filter_map(|(idx, (expr, alias))| { - if used_columns - .iter() - .map(|column| column.index()) - .collect::>() - .contains(&idx) - { - Some((expr.clone(), alias.clone())) - } else { - None - } - }) - .collect::>() -} - -fn filter_left_requirements( - required_columns: HashSet, - left_size: usize, -) -> HashSet { - required_columns - .into_iter() - .filter(|col| col.index() < left_size) - .collect() -} - /// 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 @@ -3532,17 +3482,246 @@ fn convert_projection_exprs( .collect() } -#[derive(Debug, PartialEq)] -enum RewriteState { - /// The expression is unchanged. - Unchanged, - /// Some part of the expression has been rewritten - RewrittenValid, - /// Some part of the expression has been rewritten, but some column - /// references could not be. - RewrittenInvalid, -} - +fn extend_left_mapping_with_right( + mut left_schema_mapping: HashMap, + right_child_plan: &Arc, + left_size: usize, + new_left_size: usize, +) -> HashMap { + 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. +/// +/// # Arguments +/// * `requirement_map`: Reference to a ColumnRequirements map. +/// * `column_index`: The index of the column in question. +/// +/// # Returns +/// The number of removed columns before the 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. +/// +/// 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" +/// +/// # Arguments +/// * `requirements`: Reference to a `HashSet` representing the parent's column requirements. +/// * `projection_columns`: Slice of `Column` representing the columns in the projection. +/// +/// # Returns +/// A `HashSet` with updated column indices reflecting the child's perspective. +/// +fn map_parent_reqs_to_input_reqs( + requirements: &HashSet, + projection_columns: &[Column], +) -> HashSet { + requirements + .iter() + .map(|column| projection_columns[column.index()].clone()) + .collect::>() +} + +/// Calculates the index changes of columns after the removal of a projection. +/// +/// This function iterates through the columns and records the changes in their indices +/// 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 and new index values in a hashmap. +/// +/// # Arguments +/// * `columns` - A set of columns before the projection is removed. +/// * `projection_columns` - A slice of columns as they appear in the projection. +/// +/// # Returns +/// A `HashMap` where the key is the original column and the value is the column with updated index. +fn index_changes_after_projection_removal( + columns: HashSet, + projection_columns: &[Column], +) -> HashMap { + 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() +} + +/// Filters the expressions of a [`ProjectionExec`] according to the given used column indices. +fn collect_used_columns( + projection_exprs: &[(Arc, String)], + used_columns: &HashSet, +) -> Vec<(Arc, String)> { + projection_exprs + .iter() + .enumerate() + .filter_map(|(idx, (expr, alias))| { + if used_columns + .iter() + .map(|column| column.index()) + .collect::>() + .contains(&idx) + { + Some((expr.clone(), alias.clone())) + } else { + None + } + }) + .collect::>() +} + +fn collect_left_used_columns( + required_columns: HashSet, + left_size: usize, +) -> HashSet { + required_columns + .into_iter() + .filter(|col| col.index() < left_size) + .collect() +} + +/// 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 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) +} + +/// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. +/// +/// # Arguments +/// * `plan`: Reference to an Arc of a dynamic ExecutionPlan trait object. +/// +/// # Returns +/// A `HashSet` containing all columns from the plan's schema. +fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { + plan.schema() + .fields() + .iter() + .enumerate() + .map(|(i, f)| Column::new(f.name(), i)) + .collect() +} + +/// Collects all columns 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. +/// +/// # Arguments +/// * `on`: Slice of tuples representing the equivalence conditions between columns. +/// * `filter`: Optional reference to a JoinFilter for non-equivalence conditions. +/// * `left_size`: The number of columns in the left table. +/// * `join_left_schema`: Schema reference of the left input to the join. +/// * `join_right_schema`: Schema reference of the right input to the join. +/// +/// # Returns +/// A `HashSet` containing all columns from the join conditions. +fn collect_columns_in_join_conditions( + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, + left_size: usize, + join_left_schema: SchemaRef, + join_right_schema: SchemaRef, +) -> HashSet { + let equivalence_columns = on + .iter() + .flat_map(|(col_left, col_right)| { + let left_columns = collect_columns(col_left); + let right_columns = collect_columns(col_right); + let 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() +} + +#[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`: @@ -3570,15 +3749,15 @@ fn update_expr_with_projection( .clone() .transform_up_mut(&mut |expr: Arc| { if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); } let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); + 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())) + 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; @@ -3597,12 +3776,12 @@ fn update_expr_with_projection( ) }) .map_or_else( - || Ok(Transformed::No(expr)), - |c| Ok(Transformed::Yes(c)), + || Ok(Transformed::no(expr)), + |c| Ok(Transformed::yes(c)), ) } }); - new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e)) + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e.data)) } /// Rewrites the expressions with new index values. @@ -3705,20 +3884,21 @@ fn update_column_index( .clone() .transform_up_mut(&mut |expr: Arc| { if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); } let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); }; state = RewriteState::RewrittenValid; // Update the index of `column`: if let Some(updated) = mapping.get(column) { - Ok(Transformed::Yes(Arc::new(updated.clone()) as _)) + Ok(Transformed::yes(Arc::new(updated.clone()) as _)) } else { - Ok(Transformed::No(expr.clone())) + Ok(Transformed::no(expr.clone())) } }) - .unwrap(); + .unwrap() + .data; new_expr } @@ -3737,13 +3917,13 @@ fn update_equivalence_conditions( .clone() .transform_up_mut(&mut |expr: Arc| { if left_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); } let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); }; left_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new(Column::new( + Ok(Transformed::yes(Arc::new(Column::new( column.name(), column.index() - removed_column_count( @@ -3752,18 +3932,19 @@ fn update_equivalence_conditions( ), )))) }) - .unwrap(), + .unwrap() + .data, right_col .clone() .transform_up_mut(&mut |expr: Arc| { if right_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); } let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::No(expr)); + return Ok(Transformed::no(expr)); }; right_state = RewriteState::RewrittenValid; - Ok(Transformed::Yes(Arc::new(Column::new( + Ok(Transformed::yes(Arc::new(Column::new( column.name(), column.index() - removed_column_count( @@ -3772,7 +3953,8 @@ fn update_equivalence_conditions( ), )))) }) - .unwrap(), + .unwrap() + .data, ) }) .collect() @@ -3865,203 +4047,6 @@ fn update_right_requirements( .collect() } -fn extend_left_mapping_with_right( - mut left_schema_mapping: HashMap, - right_child_plan: &Arc, - left_size: usize, - new_left_size: usize, -) -> HashMap { - 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 -} - -/// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. -/// -/// # Arguments -/// * `plan`: Reference to an Arc of a dynamic ExecutionPlan trait object. -/// -/// # Returns -/// A `HashSet` containing all columns from the plan's schema. -fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { - plan.schema() - .fields() - .iter() - .enumerate() - .map(|(i, f)| Column::new(f.name(), i)) - .collect() -} - -/// Collects all columns 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. -/// -/// # Arguments -/// * `on`: Slice of tuples representing the equivalence conditions between columns. -/// * `filter`: Optional reference to a JoinFilter for non-equivalence conditions. -/// * `left_size`: The number of columns in the left table. -/// * `join_left_schema`: Schema reference of the left input to the join. -/// * `join_right_schema`: Schema reference of the right input to the join. -/// -/// # Returns -/// A `HashSet` containing all columns from the join conditions. -fn collect_columns_in_join_conditions( - on: &[(PhysicalExprRef, PhysicalExprRef)], - filter: Option<&JoinFilter>, - left_size: usize, - join_left_schema: SchemaRef, - join_right_schema: SchemaRef, -) -> HashSet { - let equivalence_columns = on - .iter() - .flat_map(|(col_left, col_right)| { - let left_columns = collect_columns(col_left); - let right_columns = collect_columns(col_right); - let 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() -} - -/// Calculates the count of removed (unused) columns that precede a given column index. -/// -/// # Arguments -/// * `requirement_map`: Reference to a ColumnRequirements map. -/// * `column_index`: The index of the column in question. -/// -/// # Returns -/// The number of removed columns before the 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. -/// -/// 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" -/// -/// # Arguments -/// * `requirements`: Reference to a `HashSet` representing the parent's column requirements. -/// * `projection_columns`: Slice of `Column` representing the columns in the projection. -/// -/// # Returns -/// A `HashSet` with updated column indices reflecting the child's perspective. -/// -fn map_parent_reqs_to_input_reqs( - requirements: &HashSet, - projection_columns: &[Column], -) -> HashSet { - requirements - .iter() - .map(|column| projection_columns[column.index()].clone()) - .collect::>() -} - -/// Checks if all columns in the input schema are required by the projection. -/// -/// # Arguments -/// * `input_columns`: Reference to a `HashSet` representing the input columns. -/// * `projection_requires`: Reference to a `HashSet` representing the projection requirements. -/// -/// # Returns -/// `true` if all input columns are required, otherwise `false`. -/// -fn all_input_columns_required( - input_columns: &HashSet, - projection_requires: &HashSet, -) -> bool { - input_columns - .iter() - .all(|input_column| projection_requires.contains(input_column)) -} - -/// Calculates the index changes of columns after the removal of a projection. -/// -/// This function iterates through the columns and records the changes in their indices -/// 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 and new index values in a hashmap. -/// -/// # Arguments -/// * `columns` - A set of columns before the projection is removed. -/// * `projection_columns` - A slice of columns as they appear in the projection. -/// -/// # Returns -/// A `HashMap` where the key is the original column and the value is the column with updated index. -fn index_changes_after_projection_removal( - columns: HashSet, - projection_columns: &[Column], -) -> HashMap { - 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() -} - /// Rewrites a projection execution plan with updated column indices. /// /// This function updates the column indices in a projection based on a provided mapping. @@ -4206,7 +4191,6 @@ fn rewrite_hash_join( right_input_plan: Arc, left_mapping: &HashMap, right_mapping: &HashMap, - left_size: usize, ) -> Result> { let new_on = update_join_on(hj.on(), left_mapping, right_mapping); let new_filter = hj.filter().map(|filter| { @@ -4230,11 +4214,11 @@ fn rewrite_hash_join( index: right_mapping .iter() .find(|(old_column, _new_column)| { - old_column.index() == col_idx.index + left_size + old_column.index() == col_idx.index }) .map(|(_old_column, new_column)| new_column.index()) .unwrap_or(col_idx.index), - side: JoinSide::Left, + side: JoinSide::Right, }, }) .collect(), From 0aa14269f508d23a4bc0056d4489764cb0a8127d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 12 Mar 2024 12:57:16 +0300 Subject: [PATCH 31/85] Fix after merge --- .../optimize_projections.rs | 6 ++++++ .../core/src/physical_optimizer/optimizer.rs | 2 -- .../src/expressions/get_indexed_field.rs | 18 +---------------- .../physical-plan/src/joins/hash_join.rs | 2 +- .../sqllogictest/test_files/explain.slt | 3 --- datafusion/sqllogictest/test_files/joins.slt | 5 +++++ .../sqllogictest/test_files/tpch/q11.slt.part | 20 ++----------------- .../sqllogictest/test_files/tpch/q19.slt.part | 12 +++-------- .../sqllogictest/test_files/tpch/q2.slt.part | 9 +-------- .../sqllogictest/test_files/tpch/q20.slt.part | 2 +- .../sqllogictest/test_files/tpch/q5.slt.part | 10 +--------- 11 files changed, 21 insertions(+), 68 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index ae2a074ea66c..adab16ebeb06 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -1053,6 +1053,7 @@ impl ProjectionOptimizer { new_on, new_filter, hj.join_type(), + None, *hj.partition_mode(), hj.null_equals_null(), )?) as _; @@ -1091,6 +1092,7 @@ impl ProjectionOptimizer { new_on, new_filter, hj.join_type(), + None, *hj.partition_mode(), hj.null_equals_null(), )?) as _; @@ -1136,6 +1138,7 @@ impl ProjectionOptimizer { new_on, new_filter, hj.join_type(), + None, *hj.partition_mode(), hj.null_equals_null(), )?) as _; @@ -1180,6 +1183,7 @@ impl ProjectionOptimizer { new_on, new_filter, hj.join_type(), + None, *hj.partition_mode(), hj.null_equals_null(), )?) as _; @@ -1232,6 +1236,7 @@ impl ProjectionOptimizer { new_on, new_filter, hj.join_type(), + None, *hj.partition_mode(), hj.null_equals_null(), )?) as _; @@ -4231,6 +4236,7 @@ fn rewrite_hash_join( new_on, new_filter, hj.join_type(), + None, *hj.partition_mode(), hj.null_equals_null(), ) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 8f2db5ee9ad3..f23cd8d8d29b 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -101,8 +101,6 @@ impl PhysicalOptimizer { // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. Arc::new(EnforceSorting::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()), diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index 4c46babb8135..99b2279ba572 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -42,20 +42,6 @@ pub enum GetFieldAccessExpr { NamedStructField { name: ScalarValue }, } -impl GetFieldAccessExpr { - pub fn exprs(&self) -> Vec> { - match self { - GetFieldAccessExpr::NamedStructField { .. } => vec![], - GetFieldAccessExpr::ListIndex { key } => vec![key.clone()], - GetFieldAccessExpr::ListRange { - start, - stop, - stride, - } => vec![start.clone(), stop.clone(), stride.clone()], - } - } -} - impl std::fmt::Display for GetFieldAccessExpr { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { match self { @@ -178,9 +164,7 @@ impl PhysicalExpr for GetIndexedFieldExpr { } fn children(&self) -> Vec> { - let mut children = vec![self.arg.clone()]; - children.extend(self.field.exprs()); - children + vec![self.arg.clone()] } fn with_new_children( diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index a1c50a2113ba..0abf10f7ad1c 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -524,7 +524,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/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 89ab7f0d014f..d9f9fde45e93 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -248,7 +248,6 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE 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 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 PipelineChecker SAME TEXT AS ABOVE @@ -304,7 +303,6 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE 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 ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements 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]:)]] @@ -340,7 +338,6 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE 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 ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements GlobalLimitExec: skip=0, fetch=10 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index cee3a99b084f..b89647d919e8 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1718,6 +1718,11 @@ CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] ##### # Config teardown diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 0aa3c30bb6a6..3629bc151253 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -104,14 +104,7 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] --------------------------------FilterExec: n_name@1 = GERMANY ----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: n_name@1 = GERMANY ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] ------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] --------------CoalescePartitionsExec @@ -137,16 +130,7 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] --------------------------------FilterExec: n_name@1 = GERMANY ----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------FilterExec: n_name@1 = GERMANY ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query IR select diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index b71f5796b808..8b3f51eb25a7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -80,15 +80,9 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco --------------CoalesceBatchesExec: target_batch_size=8192 ----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------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 -----------------------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 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false - +--------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 +----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false query R diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index dc24080d666d..aa33680eedf7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -188,14 +188,7 @@ ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@7 as --------------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ----------------------------------------FilterExec: r_name@1 = EUROPE ------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------FilterExec: r_name@1 = EUROPE -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false query RTTITTTT diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index dc762b09ed34..85fcbf12b01d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -108,7 +108,7 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ----------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1 --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +----------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 ------------------------CoalesceBatchesExec: target_batch_size=8192 --------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] ----------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index 5decaf6993f3..cd0f5124da93 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -124,15 +124,7 @@ SortPreservingMergeExec: [revenue@1 DESC] --------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ----------------------------FilterExec: r_name@1 = ASIA ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------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 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -----------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------FilterExec: r_name@1 = ASIA -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false query TR select From fefd91f7b09b47c3b550ef2c79afbbd07ec862e6 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 12 Mar 2024 14:21:26 +0300 Subject: [PATCH 32/85] Inherit hashjoin projection --- .../optimize_projections.rs | 403 +++++++++++++++++- 1 file changed, 390 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index adab16ebeb06..d51c3489e58d 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -190,6 +190,13 @@ impl ProjectionOptimizer { no_change => no_change.data, }; + if projection_input.as_any().is::() { + self = match self.try_embed_to_hash_join()? { + join if join.transformed => return Ok(join.data), + projection => projection.data, + } + } + // Source providers: if projection_input.as_any().is::() { self = match self.try_projected_csv() { @@ -334,6 +341,81 @@ impl ProjectionOptimizer { })) } + /// 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(mut self) -> Result> { + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(self)); + }; + let Some(hash_join) = self.plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(self)); + }; + // Collect all column indices from the given projection expressions. + let projection_index = collect_column_indices(projection.expr()); + + if projection_index.is_empty() { + return Ok(Transformed::no(self)); + }; + + // 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(Transformed::no(self)); + } + + let new_hash_join = + Arc::new(hash_join.with_projection(Some(projection_index.to_vec()))?) + as Arc; + + // 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_with_projection(expr, embed_project_exprs.as_slice(), 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())?; + + 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: HashMap::new(), + children_nodes: self.children_nodes.swap_remove(0).children_nodes, + })) + } else { + let plan = Arc::new(new_projection) as Arc; + let required_columns = collect_columns_in_plan_schema(&plan); + Ok(Transformed::yes(Self { + plan, + required_columns, + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes.swap_remove(0).children_nodes, + })) + } + } + /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. fn try_projected_csv(self) -> Transformed { // These plans are known. @@ -3421,6 +3503,20 @@ fn is_expr_trivial(expr: &Arc) -> bool { || expr.as_any().downcast_ref::().is_some() } +/// 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() +} + /// 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. @@ -3716,6 +3812,20 @@ fn collect_columns_in_join_conditions( .collect() } +/// 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 +} + #[derive(Debug, PartialEq)] enum RewriteState { /// The expression is unchanged. @@ -4563,14 +4673,12 @@ fn rewrite_bounded_window_aggregate( #[cfg(test)] mod tests { + use super::*; 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::optimize_projections::{ - update_expr_with_projection, OptimizeProjections, - }; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -4580,7 +4688,7 @@ mod tests { use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - use crate::physical_plan::ExecutionPlan; + use crate::physical_plan::{get_plan_string, ExecutionPlan}; use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_common::config::ConfigOptions; @@ -4593,8 +4701,9 @@ mod tests { use datafusion_physical_expr::{ Partitioning, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }; - use datafusion_physical_plan::get_plan_string; - use datafusion_physical_plan::joins::SymmetricHashJoinExec; + use datafusion_physical_plan::joins::{ + HashJoinExec, PartitionMode, SymmetricHashJoinExec, + }; use datafusion_physical_plan::union::UnionExec; fn create_simple_csv_exec() -> Arc { @@ -4605,6 +4714,7 @@ mod tests { Field::new("d", DataType::Int32, true), Field::new("e", DataType::Int32, true), ])); + Arc::new(CsvExec::new( FileScanConfig { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), @@ -4632,6 +4742,7 @@ mod tests { Field::new("d", DataType::Int32, true), Field::new("e", DataType::Int32, true), ])); + Arc::new(CsvExec::new( FileScanConfig { object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), @@ -4719,6 +4830,7 @@ mod tests { (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)), @@ -4777,11 +4889,13 @@ mod tests { ))), )?), ]; + 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(()) } @@ -4846,13 +4960,14 @@ mod tests { )?), ]; let projected_exprs: Vec<(Arc, String)> = vec![ - (Arc::new(Column::new("a", 0)), "a".to_owned()), + (Arc::new(Column::new("a", 3)), "a".to_owned()), (Arc::new(Column::new("b", 1)), "b_new".to_owned()), - (Arc::new(Column::new("c", 2)), "c".to_owned()), - (Arc::new(Column::new("d", 3)), "d_new".to_owned()), - (Arc::new(Column::new("e", 4)), "e".to_owned()), - (Arc::new(Column::new("f", 5)), "f_new".to_owned()), + (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)), @@ -4911,11 +5026,29 @@ mod tests { ))), )?), ]; + 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_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(()) } @@ -4935,12 +5068,15 @@ mod tests { " 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(()) } @@ -4971,6 +5107,7 @@ mod tests { ], 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]", @@ -4978,13 +5115,16 @@ mod tests { " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; assert_eq!(initial, expected_initial); + let after_optimize = OptimizeProjections::new().optimize(top_projection, &ConfigOptions::new())?; + let expected = [ "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) } @@ -5008,14 +5148,17 @@ mod tests { " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_eq!(initial, expected_initial); + let after_optimize = OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; + let expected = [ "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@2 as d]", " CoalescePartitionsExec", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false", ]; assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) } @@ -5045,6 +5188,7 @@ mod tests { ], 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]", @@ -5052,6 +5196,7 @@ mod tests { " 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())?; @@ -5059,7 +5204,9 @@ mod tests { "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(()) } @@ -5067,6 +5214,7 @@ mod tests { 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, @@ -5134,15 +5282,18 @@ mod tests { " 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]", + "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, @@ -5157,6 +5308,7 @@ mod tests { side: JoinSide::Left, }, ]; + assert_eq!( expected_filter_col_ind, after_optimize.children()[0] @@ -5167,6 +5319,205 @@ mod tests { .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<()> { + // 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 = + 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@4 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", + " 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 = [ + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c, c@4 as c]", + " 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", + " 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(()) } @@ -5199,15 +5550,17 @@ mod tests { " 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(()) } @@ -5239,6 +5592,7 @@ mod tests { ], 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]", @@ -5246,6 +5600,7 @@ mod tests { " 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())?; @@ -5255,6 +5610,7 @@ mod tests { " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) } @@ -5286,6 +5642,7 @@ mod tests { ], 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]", @@ -5293,6 +5650,7 @@ mod tests { " 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())?; @@ -5302,6 +5660,7 @@ mod tests { " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" ]; assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) } @@ -5318,6 +5677,7 @@ mod tests { ], 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]", @@ -5327,6 +5687,7 @@ mod tests { " 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 = [ @@ -5336,11 +5697,14 @@ mod tests { " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" ]; + assert_eq!(get_plan_string(&after_optimize), expected); + Ok(()) } #[test] + fn test_optimize_projections_filter_sort() -> Result<()> { /* INITIAL PLAN: @@ -5359,7 +5723,9 @@ mod tests { ProjectionExec: |c@2 |a@0 as x | CsvExec: |a |b |c |d |e | */ + let csv = create_simple_csv_exec(); + let projection1 = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("c", 2)), "c".to_string()), @@ -5368,6 +5734,7 @@ mod tests { ], csv, )?); + let projection2 = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("x", 1)), "x".to_string()), @@ -5376,6 +5743,7 @@ mod tests { ], projection1, )?); + let sort = Arc::new(SortExec::new( vec![ PhysicalSortExpr { @@ -5389,6 +5757,7 @@ mod tests { ], projection2, )); + let projection3 = Arc::new(ProjectionExec::try_new( vec![ (Arc::new(Column::new("x", 2)), "x".to_string()), @@ -5397,6 +5766,7 @@ mod tests { ], sort, )?); + let projection4 = Arc::new(ProjectionExec::try_new( vec![( Arc::new(BinaryExpr::new( @@ -5408,6 +5778,7 @@ mod tests { )], projection3, )?); + let filter = Arc::new(FilterExec::try_new( Arc::new(BinaryExpr::new( Arc::new(Column::new("sum", 0)), @@ -5416,7 +5787,9 @@ mod tests { )), projection4, )?) as Arc; + let initial = get_plan_string(&filter); + let expected_initial = [ "FilterExec: sum@0 > 0", " ProjectionExec: expr=[c@2 + x@0 as sum]", @@ -5425,15 +5798,19 @@ mod tests { " ProjectionExec: expr=[x@1 as x, c@0 as c, a@2 as x]", " ProjectionExec: expr=[c@2 as c, e@4 as x, a@0 as a]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(initial, expected_initial); + let after_optimize = OptimizeProjections::new().optimize(filter, &ConfigOptions::new())?; + let expected = [ "FilterExec: sum@0 > 0", " ProjectionExec: expr=[c@0 + x@1 as sum]", " SortExec: expr=[c@0 ASC,x@1 ASC]", " ProjectionExec: expr=[c@2 as c, a@0 as x]", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) } From 464d7bbecfaa1e5162228273f7910d6241b4ae83 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 12 Mar 2024 16:26:44 +0300 Subject: [PATCH 33/85] Minor changes --- .../optimize_projections.rs | 95 +++++++++++-------- 1 file changed, 54 insertions(+), 41 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index d51c3489e58d..d81359fe5487 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -153,7 +153,7 @@ impl ProjectionOptimizer { /// 1. If the input plan is also a projection, they can be merged into one projection. /// 2. The projection can be removed. /// 3. The projection can get narrower. - /// 4. The projection can be embedded into the source. + /// 4. The projection can be 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]; @@ -184,15 +184,18 @@ impl ProjectionOptimizer { // 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 source provider. + // 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::() { self = match self.try_embed_to_hash_join()? { - join if join.transformed => return Ok(join.data), + join if join.transformed => { + return join.data.adjust_node_with_requirements() + } projection => projection.data, } } @@ -204,7 +207,7 @@ impl ProjectionOptimizer { no_change => no_change.data, } } - // TODO: Other source execs can be implemented here if projection can be applied within the scope of them. + // 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. @@ -341,8 +344,9 @@ impl ProjectionOptimizer { })) } - /// 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. + /// 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> { let Some(projection) = self.plan.as_any().downcast_ref::() else { return Ok(Transformed::no(self)); @@ -350,17 +354,15 @@ impl ProjectionOptimizer { let Some(hash_join) = self.plan.as_any().downcast_ref::() else { return Ok(Transformed::no(self)); }; + // Collect all column indices from the given projection expressions. let projection_index = collect_column_indices(projection.expr()); - if projection_index.is_empty() { - return Ok(Transformed::no(self)); - }; - // 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() + if projection_index.is_empty() + || (projection_index.len() == projection_index.last().unwrap() + 1 + && projection_index.len() == hash_join.schema().fields().len()) { return Ok(Transformed::no(self)); } @@ -382,7 +384,6 @@ impl ProjectionOptimizer { .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) = @@ -392,28 +393,37 @@ impl ProjectionOptimizer { }; 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. + // 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())?; - if is_projection_removable(&new_projection) { + let new_node = if is_projection_removable(&new_projection) { let required_columns = collect_columns_in_plan_schema(&new_hash_join); - Ok(Transformed::yes(Self { + Self { plan: new_hash_join, required_columns, schema_mapping: HashMap::new(), children_nodes: self.children_nodes.swap_remove(0).children_nodes, - })) + } } else { + let new_join_node = Self { + plan: new_hash_join, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: self.children_nodes.swap_remove(0).children_nodes, + }; let plan = Arc::new(new_projection) as Arc; let required_columns = collect_columns_in_plan_schema(&plan); - Ok(Transformed::yes(Self { + Self { plan, required_columns, schema_mapping: HashMap::new(), - children_nodes: self.children_nodes.swap_remove(0).children_nodes, - })) - } + children_nodes: vec![new_join_node], + } + }; + Ok(Transformed::yes(new_node)) } /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. @@ -505,7 +515,7 @@ impl ProjectionOptimizer { else if let Some(cj) = plan.as_any().downcast_ref::() { self = self.try_insert_below_cross_join(cj)? } - // Specially handled joins and aggregations + // Joins and aggregations require special attention. else if let Some(hj) = plan.as_any().downcast_ref::() { self = self.try_insert_below_hash_join(hj)? } else if let Some(nlj) = plan.as_any().downcast_ref::() { @@ -515,22 +525,14 @@ impl ProjectionOptimizer { } else if let Some(shj) = plan.as_any().downcast_ref::() { self = self.try_insert_below_symmetric_hash_join(shj)? } else if let Some(agg) = plan.as_any().downcast_ref::() { - if agg.aggr_expr().iter().any(|expr| { - expr.clone() - .with_new_expressions(expr.expressions()) - .is_none() - }) { + 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 = self.try_insert_below_aggregate(agg)? } else if let Some(w_agg) = plan.as_any().downcast_ref::() { - if w_agg.window_expr().iter().any(|expr| { - expr.clone() - .with_new_expressions(expr.expressions()) - .is_none() - }) { + if !is_window_expr_rewritable(w_agg.window_expr()) { self.children_nodes[0].required_columns = collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(self); @@ -538,19 +540,12 @@ impl ProjectionOptimizer { self = self.try_insert_below_window_aggregate(w_agg)? } else if let Some(bw_agg) = plan.as_any().downcast_ref::() { - if bw_agg.window_expr().iter().any(|expr| { - expr.clone() - .with_new_expressions(expr.expressions()) - .is_none() - }) { + if !is_window_expr_rewritable(bw_agg.window_expr()) { self.children_nodes[0].required_columns = collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(self); } self = self.try_insert_below_bounded_window_aggregate(bw_agg)? - } else if let Some(_file_sink) = plan.as_any().downcast_ref::() { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan) } else { self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan) @@ -3517,6 +3512,24 @@ fn is_projection_removable(projection: &ProjectionExec) -> bool { }) && 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().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_some() + }) +} + +/// Tries to rewrite the [`WindowExpr`] with the existing expressions to keep on optimization. +fn is_window_expr_rewritable(window_expr: &[Arc<(dyn WindowExpr)>]) -> bool { + window_expr.iter().any(|expr| { + expr.clone() + .with_new_expressions(expr.expressions()) + .is_some() + }) +} + /// 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. @@ -3819,7 +3832,7 @@ fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec>() + .collect::>() .into_iter() .collect::>(); indexs.sort(); From ca5d843be40b5aa7f0abfa55c8e70bde390091ce Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 13 Mar 2024 15:06:48 +0300 Subject: [PATCH 34/85] Project csv only with requirements --- .../optimize_projections.rs | 154 +++++++++++++----- 1 file changed, 117 insertions(+), 37 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index d81359fe5487..eacee70b9eb0 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -202,10 +202,7 @@ impl ProjectionOptimizer { // Source providers: if projection_input.as_any().is::() { - self = match self.try_projected_csv() { - new_csv if new_csv.transformed => return Ok(new_csv.data), - no_change => no_change.data, - } + return self.try_projected_csv(); } // TODO: Other source execs can be implemented here if projection can be applied inside them. @@ -427,45 +424,128 @@ impl ProjectionOptimizer { } /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. - fn try_projected_csv(self) -> Transformed { + fn try_projected_csv(self) -> Result { // These plans are known. let Some(projection) = self.plan.as_any().downcast_ref::() else { - return Transformed::no(self); + return Ok(self); }; let Some(csv) = projection.input().as_any().downcast_ref::() else { - return Transformed::no(self); + return Ok(self); }; // 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. - let Some(projection_columns) = collect_alias_free_columns(projection.expr()) - else { - return Transformed::no(self); - }; + if let Some(projection_columns) = collect_alias_free_columns(projection.expr()) { + let mut file_scan = csv.base_config().clone(); + let new_projections = new_projections_for_columns( + &projection_columns, + &file_scan + .projection + .unwrap_or((0..csv.schema().fields().len()).collect()), + ); - let mut file_scan = csv.base_config().clone(); + 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, + )) as _, + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), // Sources cannot have a mapping. + children_nodes: vec![], + }) + } else { + let mut file_scan = csv.base_config().clone(); + let used_column_indices = projection + .expr() + .iter() + .flat_map(|(expr, _)| collect_columns(expr)) + .collect::>() + .into_iter() + .map(|col| col.index()) + .collect::>(); - let new_projections = new_projections_for_columns( - &projection_columns, - &file_scan + let used_csv_indices = file_scan .projection - .unwrap_or((0..csv.schema().fields().len()).collect()), - ); + .clone() + .unwrap_or((0..csv.schema().fields().len()).collect()) + .iter() + .enumerate() + .filter_map(|(idx, csv_indx)| { + if used_column_indices.contains(&idx) { + Some(*csv_indx) + } else { + None + } + }) + .collect::>(); - file_scan.projection = Some(new_projections); - - Transformed::yes(ProjectionOptimizer { - plan: Arc::new(CsvExec::new( - file_scan, - csv.has_header(), - csv.delimiter(), - csv.quote(), - csv.escape(), - csv.file_compression_type, - )) as _, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), // Sources cannot have a mapping. - children_nodes: vec![], - }) + let unused_indices = file_scan + .projection + .unwrap_or((0..csv.schema().fields().len()).collect()) + .iter() + .enumerate() + .filter_map(|(idx, csv_idx)| { + if used_csv_indices.contains(&csv_idx) { + None + } else { + Some(idx) + } + }) + .collect::>(); + + let new_exprs = projection + .expr() + .iter() + .map(|(expr, alias)| { + let new_expr = expr + .clone() + .transform_up_mut(&mut |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::>>()?; + + file_scan.projection = Some(used_csv_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, + )) as _, + required_columns: HashSet::new(), + schema_mapping: HashMap::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 @@ -5133,9 +5213,9 @@ mod tests { OptimizeProjections::new().optimize(top_projection, &ConfigOptions::new())?; let expected = [ - "ProjectionExec: expr=[b@1 as new_b, c@2 + e@4 as binary, b@1 as newest_b]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false" - ]; + "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(()) @@ -5821,8 +5901,8 @@ mod tests { "FilterExec: sum@0 > 0", " ProjectionExec: expr=[c@0 + x@1 as sum]", " SortExec: expr=[c@0 ASC,x@1 ASC]", - " ProjectionExec: expr=[c@2 as c, a@0 as x]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false"]; + " 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(()) From b586311cbc12a06383693272603305c2fbb05705 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 15 Mar 2024 09:46:49 +0300 Subject: [PATCH 35/85] to check diff --- datafusion/core/src/dataframe/mod.rs | 3 +- .../optimize_projections.rs | 357 +++++++++++++++--- datafusion/core/tests/sql/explain_analyze.rs | 23 +- 3 files changed, 317 insertions(+), 66 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index 3bdf2af4552d..a30fee30499f 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2933,11 +2933,12 @@ mod tests { for join_type in all_join_types { let join = left.clone().join( right.clone(), - join_type, + join_type.clone(), &["c1", "c2"], &["c2_c1", "c2_c2"], None, )?; + println!("join_type: {:?}", join_type); let physical_plan = join.create_physical_plan().await?; let out_partitioning = physical_plan.output_partitioning(); let join_schema = physical_plan.schema(); diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index eacee70b9eb0..aa318ca48c17 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -79,6 +79,7 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::displayable; use datafusion_physical_plan::insert::FileSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, @@ -96,6 +97,13 @@ use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use itertools::Itertools; +fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) +} + /// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary /// fields for column requirements and changed indices of columns. #[derive(Debug, Clone)] @@ -192,12 +200,13 @@ impl ProjectionOptimizer { // HashJoinExec can own the projection above. if projection_input.as_any().is::() { - self = match self.try_embed_to_hash_join()? { + return match self.try_embed_to_hash_join()? { join if join.transformed => { - return join.data.adjust_node_with_requirements() + // Re-run on the new HashJoin node + join.data.adjust_node_with_requirements() } - projection => projection.data, - } + projection => Ok(projection.data), + }; } // Source providers: @@ -348,7 +357,8 @@ impl ProjectionOptimizer { let Some(projection) = self.plan.as_any().downcast_ref::() else { return Ok(Transformed::no(self)); }; - let Some(hash_join) = self.plan.as_any().downcast_ref::() else { + let Some(hash_join) = projection.input().as_any().downcast_ref::() + else { return Ok(Transformed::no(self)); }; @@ -396,31 +406,30 @@ impl ProjectionOptimizer { let new_projection = ProjectionExec::try_new(new_projection_exprs, new_hash_join.clone())?; - let new_node = if is_projection_removable(&new_projection) { + if is_projection_removable(&new_projection) { let required_columns = collect_columns_in_plan_schema(&new_hash_join); - Self { + Ok(Transformed::yes(Self { plan: new_hash_join, required_columns, schema_mapping: HashMap::new(), children_nodes: self.children_nodes.swap_remove(0).children_nodes, - } + })) } else { + let required_columns = collect_columns_in_plan_schema(&new_hash_join); let new_join_node = Self { plan: new_hash_join, - required_columns: HashSet::new(), + required_columns, schema_mapping: HashMap::new(), children_nodes: self.children_nodes.swap_remove(0).children_nodes, }; let plan = Arc::new(new_projection) as Arc; - let required_columns = collect_columns_in_plan_schema(&plan); - Self { + Ok(Transformed::no(Self { plan, - required_columns, + required_columns: HashSet::new(), schema_mapping: HashMap::new(), children_nodes: vec![new_join_node], - } - }; - Ok(Transformed::yes(new_node)) + })) + } } /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. @@ -490,7 +499,7 @@ impl ProjectionOptimizer { .iter() .enumerate() .filter_map(|(idx, csv_idx)| { - if used_csv_indices.contains(&csv_idx) { + if used_csv_indices.contains(csv_idx) { None } else { Some(idx) @@ -1035,12 +1044,8 @@ impl ProjectionOptimizer { ) -> 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, - ); + let (analyzed_join_left, analyzed_join_right) = + analyze_requirements_of_joins(cj.schema(), &self.required_columns, left_size); match ( all_columns_required(&analyzed_join_left), all_columns_required(&analyzed_join_right), @@ -1155,6 +1160,256 @@ impl ProjectionOptimizer { mut self, hj: &HashJoinExec, ) -> Result { + if matches!( + hj.join_type, + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full + ) { + 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 mut hj_left_requirements = self + .required_columns + .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( + hj.on() + .iter() + .flat_map(|(left_on, _)| collect_columns(left_on)) + .collect::>(), + ); + hj_left_requirements.extend( + hj.filter() + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Left { + Some(Column::new( + hj.left().schema().fields()[col_ind.index].name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + + let mut hj_right_requirements = self + .required_columns + .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( + hj.on() + .iter() + .flat_map(|(_, right_on)| collect_columns(right_on)) + .collect::>(), + ); + hj_right_requirements.extend( + hj.filter() + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Right { + Some(Column::new( + hj.right().schema().fields()[col_ind.index] + .name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + + 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 left_projection_exprs = hj_left_requirements + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[0].clone()], + }; + + let right_projection_exprs = hj_right_requirements + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[1].clone()], + }; + + 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.into_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::>(); + + 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_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }); + + let new_projection = hj.projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj.left().schema().fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }); + println!("new_projection: {:?}", new_projection); + + 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(), + )?; + return Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }); + } + (false, true) => {} + (true, false) => {} + (true, true) => {} + } + } let left_size = hj.left().schema().fields().len(); // HashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. match hj.join_type() { @@ -1173,12 +1428,8 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); - let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - hj.left(), - hj.right(), - &self.required_columns, - left_size, - ); + let (analyzed_join_left, analyzed_join_right) = + analyze_requirements_of_joins(hj.schema(), &self.required_columns, left_size); match hj.join_type() { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { @@ -1449,8 +1700,7 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - nlj.left(), - nlj.right(), + nlj.schema(), &self.required_columns, left_size, ); @@ -1677,8 +1927,7 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - smj.left(), - smj.right(), + smj.schema(), &self.required_columns, left_size, ); @@ -1944,8 +2193,7 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - shj.left(), - shj.right(), + shj.schema(), &self.required_columns, left_size, ); @@ -3327,6 +3575,8 @@ impl PhysicalOptimizerRule for OptimizeProjections { plan: Arc, _config: &ConfigOptions, ) -> Result> { + print_plan(&plan); + // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); let mut optimizer = ProjectionOptimizer::new_default(plan); @@ -3455,18 +3705,16 @@ fn analyze_requirements(node: &ProjectionOptimizer) -> ColumnRequirements { /// - The second element represents the column requirements for the right child. /// fn analyze_requirements_of_joins( - left_child: &Arc, - right_child: &Arc, + schema: SchemaRef, required_columns: &HashSet, 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 columns_in_schema = schema + .fields() + .iter() + .enumerate() + .map(|(idx, field)| Column::new(field.name(), idx)) + .collect::>(); let requirement_map = columns_in_schema .into_iter() .map(|col| { @@ -3902,10 +4150,10 @@ fn collect_columns_in_join_conditions( equivalence_columns .into_iter() .chain(non_equivalence_columns) - .collect() + .collect::>() } -/// Collect all column indices from the given projection expressions. +/// Collect all column indices from the given projection expressions in the index order. fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec { // Collect indices and remove duplicates. let mut indexs = exprs @@ -3976,7 +4224,7 @@ fn update_expr_with_projection( .find_map(|(index, (projected_expr, alias))| { projected_expr.as_any().downcast_ref::().and_then( |projected_column| { - column.name().eq(projected_column.name()).then(|| { + column.eq(projected_column).then(|| { state = RewriteState::RewrittenValid; Arc::new(Column::new(alias, index)) as _ }) @@ -4442,8 +4690,9 @@ fn rewrite_hash_join( None, *hj.partition_mode(), hj.null_equals_null(), - ) - .map(|plan| Arc::new(plan) as _) + )? + .with_projection(hj.projection.clone()) + .map(|hj| Arc::new(hj) as _) } fn rewrite_nested_loop_join( @@ -5510,7 +5759,8 @@ mod tests { #[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 + // 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(); @@ -5584,10 +5834,11 @@ mod tests { 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@4 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", - " 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"]; + 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( diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index fa160f2a117e..0d2d3b22c205 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -656,18 +656,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", - " ProjectionExec: expr=[c1@0 as c1]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", - " ProjectionExec: expr=[c1@0 as c2]", - " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true" + "CoalesceBatchesExec: target_batch_size=4096", + " 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(); From 0bdd3b750b2b9150ce1ad20c168c760ef12e77d3 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 18 Mar 2024 16:54:14 +0300 Subject: [PATCH 36/85] too many open files! --- datafusion/core/src/dataframe/mod.rs | 2 +- .../optimize_projections.rs | 1184 ++++++++++++++++- .../sqllogictest/test_files/group_by.slt | 33 +- datafusion/sqllogictest/test_files/join.slt | 2 +- .../join_disable_repartition_joins.slt | 22 +- datafusion/sqllogictest/test_files/joins.slt | 228 ++-- .../sqllogictest/test_files/predicates.slt | 62 +- .../sqllogictest/test_files/tpch/q10.slt.part | 57 +- .../sqllogictest/test_files/tpch/q11.slt.part | 76 +- .../sqllogictest/test_files/tpch/q12.slt.part | 22 +- .../sqllogictest/test_files/tpch/q13.slt.part | 23 +- .../sqllogictest/test_files/tpch/q14.slt.part | 23 +- .../sqllogictest/test_files/tpch/q15.slt.part | 64 +- .../sqllogictest/test_files/tpch/q16.slt.part | 21 +- .../sqllogictest/test_files/tpch/q17.slt.part | 38 +- .../sqllogictest/test_files/tpch/q18.slt.part | 32 +- .../sqllogictest/test_files/tpch/q19.slt.part | 27 +- .../sqllogictest/test_files/tpch/q2.slt.part | 174 --- datafusion/sqllogictest/test_files/union.slt | 29 +- 19 files changed, 1469 insertions(+), 650 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index a30fee30499f..20dacf5c4ee1 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2933,7 +2933,7 @@ mod tests { for join_type in all_join_types { let join = left.clone().join( right.clone(), - join_type.clone(), + join_type, &["c1", "c2"], &["c2_c1", "c2_c2"], None, diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index aa318ca48c17..64bd857f2f36 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -139,6 +139,9 @@ impl ProjectionOptimizer { /// 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 { + // println!("1)NODE"); + // print_plan(&self.plan); + // println!("reqs: {:?}", self.required_columns); // 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 @@ -147,14 +150,19 @@ impl ProjectionOptimizer { return Ok(self); } - if self.plan.as_any().is::() { + let x: ProjectionOptimizer = 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() - } + }?; + // println!("2)NODE"); + // print_plan(&x.plan); + // println!("reqs: {:?}", x.children_nodes[0].required_columns); + // println!("maps: {:?}", x.schema_mapping); + Ok(x) } /// The function tries 4 cases: @@ -168,7 +176,7 @@ impl ProjectionOptimizer { // We first need to check having 2 sequential projections in case of merging them. if projection_input.as_any().is::() { - self = match self.try_unifying_projections()? { + 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. @@ -233,7 +241,7 @@ impl ProjectionOptimizer { } /// If it is beneficial. unifies the projection and its input, which is also a [`ProjectionExec`]. - fn try_unifying_projections(mut self) -> Result> { + 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)); @@ -260,9 +268,12 @@ impl ProjectionOptimizer { projected_exprs.push((expr, alias.clone())); } - let new_plan = - ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) - .map(|e| Arc::new(e) as _)?; + let new_plan = ProjectionExec::try_new( + projected_exprs.clone(), + child_projection.input().clone(), + ) + .map(|e| Arc::new(e) as _)?; + Ok(Transformed::yes(Self { plan: new_plan, // Schema of the projection does not change, @@ -283,6 +294,7 @@ impl ProjectionOptimizer { else { return Ok(Transformed::no(self)); }; + // The projection must have all column expressions without aliases. let Some(projection_columns) = collect_alias_free_columns(projection_exec.expr()) else { @@ -301,6 +313,7 @@ impl ProjectionOptimizer { &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, @@ -363,7 +376,7 @@ impl ProjectionOptimizer { }; // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices(projection.expr()); + let projection_index = collect_column_indices_hj(projection.expr()); // 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. @@ -371,6 +384,8 @@ impl ProjectionOptimizer { || (projection_index.len() == projection_index.last().unwrap() + 1 && projection_index.len() == hash_join.schema().fields().len()) { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(Transformed::no(self)); } @@ -396,6 +411,8 @@ impl ProjectionOptimizer { let Some(expr) = update_expr_with_projection(expr, embed_project_exprs.as_slice(), false)? else { + self.children_nodes[0].required_columns = + collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(Transformed::no(self)); }; new_projection_exprs.push((expr, alias.clone())); @@ -1044,8 +1061,12 @@ impl ProjectionOptimizer { ) -> 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.schema(), &self.required_columns, left_size); + 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), @@ -1206,7 +1227,6 @@ impl ProjectionOptimizer { }) .unwrap_or_default(), ); - let mut hj_right_requirements = self .required_columns .iter() @@ -1261,7 +1281,731 @@ impl ProjectionOptimizer { match (keep_left_same, keep_right_same) { (false, false) => { - let left_projection_exprs = hj_left_requirements + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[0].clone()], + }; + + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[1].clone()], + }; + + 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.into_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::>(); + + 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_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }); + + let new_projection = hj.projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj.left().schema().fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }); + if HashJoinExec::try_new( + new_left_node.plan.clone(), + new_right_node.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + new_projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_node.plan); + print_plan(&new_right_node.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", new_projection); + } + + 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(), + )?; + return Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }); + } + (false, true) => { + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[0].clone()], + }; + + 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.into_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::>(); + + 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_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }); + + let new_projection = hj.projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj.left().schema().fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }); + if HashJoinExec::try_new( + new_left_node.plan.clone(), + hj.right().clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + new_projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_node.plan); + print_plan(&hj.right()); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", new_projection); + } + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + hj.right().clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + return Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![ + new_left_node, + self.children_nodes[1].clone(), + ], + }); + } + (true, false) => { + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[1].clone()], + }; + + 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.into_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::>(); + + 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_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }); + + let new_projection = hj.projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj.left().schema().fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }); + if HashJoinExec::try_new( + hj.left().clone(), + new_right_node.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + new_projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&hj.left()); + print_plan(&new_right_node.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", new_projection); + } + let new_hash_join = HashJoinExec::try_new( + hj.left().clone(), + new_right_node.plan.clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + return Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![ + self.children_nodes[0].clone(), + new_right_node, + ], + }); + } + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }); + return Ok(self); + } + } + } + if matches!(hj.join_type, JoinType::LeftAnti | JoinType::LeftSemi) { + let join_left_input_size = hj.left().schema().fields().len(); + let join_projection = hj + .projection + .clone() + .unwrap_or((0..hj.left().schema().fields().len()).collect()); + let mut hj_left_requirements = self + .required_columns + .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( + hj.on() + .iter() + .flat_map(|(left_on, _)| collect_columns(left_on)) + .collect::>(), + ); + hj_left_requirements.extend( + hj.filter() + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Left { + Some(Column::new( + hj.left().schema().fields()[col_ind.index].name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + let mut hj_right_requirements = HashSet::new(); + hj_right_requirements.extend( + hj.on() + .iter() + .flat_map(|(_, right_on)| collect_columns(right_on)) + .collect::>(), + ); + hj_right_requirements.extend( + hj.filter() + .map(|filter| { + filter + .column_indices() + .iter() + .filter_map(|col_ind| { + if col_ind.side == JoinSide::Right { + Some(Column::new( + hj.right().schema().fields()[col_ind.index] + .name(), + col_ind.index, + )) + } else { + None + } + }) + .collect::>() + }) + .unwrap_or_default(), + ); + + 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 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[0].clone()], + }; + + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![self.children_nodes[1].clone()], + }; + + 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.into_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::>(); + + 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_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }); + + let new_projection = hj.projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj.left().schema().fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }); + if HashJoinExec::try_new( + new_left_node.plan.clone(), + new_right_node.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + new_projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_node.plan); + print_plan(&new_right_node.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", new_projection); + } + 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(), + )?; + return Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }); + } + (false, true) => { + 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 .iter() .map(|req| { let name = req.name().to_owned(); @@ -1283,7 +2027,131 @@ impl ProjectionOptimizer { children_nodes: vec![self.children_nodes[0].clone()], }; - let right_projection_exprs = hj_right_requirements + 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.into_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::>(); + + 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_ind| { + if col_ind.side == JoinSide::Left { + ColumnIndex { + index: left_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == col_ind.index + }) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }); + + let new_projection = hj.projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj.left().schema().fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }); + if HashJoinExec::try_new( + new_left_node.plan.clone(), + hj.right().clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + new_projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_node.plan); + print_plan(&hj.right()); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", new_projection); + } + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + hj.right().clone(), + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + )?; + return Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![ + new_left_node, + self.children_nodes[1].clone(), + ], + }); + } + (true, false) => { + 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 .iter() .map(|req| { let name = req.name().to_owned(); @@ -1386,10 +2254,27 @@ impl ProjectionOptimizer { }) .collect() }); - println!("new_projection: {:?}", new_projection); - + if HashJoinExec::try_new( + hj.left().clone(), + new_right_node.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + new_projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&hj.left()); + print_plan(&new_right_node.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", new_projection); + } let new_hash_join = HashJoinExec::try_new( - new_left_node.plan.clone(), + hj.left().clone(), new_right_node.plan.clone(), new_on, new_filter, @@ -1402,12 +2287,19 @@ impl ProjectionOptimizer { plan: Arc::new(new_hash_join), required_columns: HashSet::new(), schema_mapping: HashMap::new(), - children_nodes: vec![new_left_node, new_right_node], + children_nodes: vec![ + self.children_nodes[0].clone(), + new_right_node, + ], + }); + } + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); }); + return Ok(self); } - (false, true) => {} - (true, false) => {} - (true, true) => {} } } let left_size = hj.left().schema().fields().len(); @@ -1428,8 +2320,12 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); - let (analyzed_join_left, analyzed_join_right) = - analyze_requirements_of_joins(hj.schema(), &self.required_columns, left_size); + let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( + hj.left(), + hj.right(), + &self.required_columns, + left_size, + ); match hj.join_type() { JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { @@ -1455,6 +2351,25 @@ impl ProjectionOptimizer { analyzed_join_left, analyzed_join_right, )?; + if HashJoinExec::try_new( + new_left_child.plan.clone(), + new_right_child.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + None, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_child.plan); + print_plan(&new_right_child.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :None"); + } let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), new_right_child.plan.clone(), @@ -1493,7 +2408,25 @@ impl ProjectionOptimizer { right_child.required_columns = update_right_child_requirements(&required_columns, left_size); - + if HashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + None, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_child.plan); + print_plan(&right_child.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :None",); + } let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -1539,7 +2472,25 @@ impl ProjectionOptimizer { left_child.required_columns = collect_left_used_columns(required_columns, left_size); - + if HashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + None, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&left_child.plan); + print_plan(&new_right_child.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :None",); + } let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1584,7 +2535,25 @@ impl ProjectionOptimizer { let (new_left_child, left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; - + if HashJoinExec::try_new( + new_left_child.plan.clone(), + right_child.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + None, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&new_left_child.plan); + print_plan(&right_child.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :None",); + } let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -1637,7 +2606,25 @@ impl ProjectionOptimizer { ); let (new_right_child, right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; - + if HashJoinExec::try_new( + left_child.plan.clone(), + new_right_child.plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + None, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&left_child.plan); + print_plan(&new_right_child.plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :None",); + } let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1700,7 +2687,8 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - nlj.schema(), + nlj.left(), + nlj.right(), &self.required_columns, left_size, ); @@ -1927,7 +2915,8 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - smj.schema(), + smj.left(), + smj.right(), &self.required_columns, left_size, ); @@ -2193,7 +3182,8 @@ impl ProjectionOptimizer { self.children_nodes[1].plan.schema(), )); let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - shj.schema(), + shj.left(), + shj.right(), &self.required_columns, left_size, ); @@ -3170,6 +4160,7 @@ impl ProjectionOptimizer { self.children_nodes[0].plan.clone(), &all_mappings[0], )?; + // Rewriting the projection does not change its output schema, // and projections does not need to transfer the mapping to upper nodes. } else if let Some(filter) = plan_any.downcast_ref::() { @@ -3203,7 +4194,14 @@ impl ProjectionOptimizer { )?; update_mapping(&mut self, all_mappings) } else if let Some(hj) = plan_any.downcast_ref::() { - let left_size = self.children_nodes[0].plan.schema().fields().len(); + let projection = hj.projection.clone(); + let left_input_size = self.children_nodes[0].plan.schema().fields().len(); + let left_size = projection + .clone() + .unwrap_or((0..hj.schema().fields().len()).collect()) + .iter() + .filter(|idx| **idx < left_input_size) + .count(); let left_mapping = all_mappings.swap_remove(0); let right_mapping = all_mappings.swap_remove(0); let new_mapping = left_mapping @@ -3216,34 +4214,19 @@ impl ProjectionOptimizer { ) })) .collect::>(); + self.plan = rewrite_hash_join( hj, self.children_nodes[0].plan.clone(), self.children_nodes[1].plan.clone(), &left_mapping, &right_mapping, + projection, + new_mapping, )?; - match hj.join_type() { - JoinType::Right - | JoinType::Full - | JoinType::Left - | JoinType::Inner => { - let (new_left, new_right) = - new_mapping.into_iter().partition(|(col_initial, _)| { - col_initial.index() < left_size - }); - all_mappings.push(new_left); - all_mappings[0].extend(new_right); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - all_mappings.push(left_mapping) - } - JoinType::RightAnti | JoinType::RightSemi => { - all_mappings.push(right_mapping) - } - }; - // self.schema_mapping = all_mappings; - update_mapping(&mut self, all_mappings) + + // self.schema_mapping = empty; + self.schema_mapping = HashMap::new(); } else if let Some(nlj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); let left_mapping = all_mappings.swap_remove(0); @@ -3475,6 +4458,7 @@ impl ProjectionOptimizer { 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) = collect_alias_free_columns(projection.expr()) @@ -3527,6 +4511,7 @@ impl ProjectionOptimizer { .collect() } } + Ok(self) } } @@ -3543,6 +4528,13 @@ impl ConcreteTreeNode for ProjectionOptimizer { fn with_new_children(mut self, children: Vec) -> Result { self.children_nodes = children; + // println!("INDEX UPDATE"); + // print_plan(&self.plan); + // self.children_nodes.iter().for_each(|c| { + // print_plan(&c.plan); + // println!("MAP: {:?}", c.schema_mapping); + // }); + self = match self.index_updater()? { new_node if new_node.transformed => new_node.data, same_node => ProjectionOptimizer::new_default(same_node.data.plan), @@ -3553,6 +4545,7 @@ impl ConcreteTreeNode for ProjectionOptimizer { // 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) @@ -3575,10 +4568,11 @@ impl PhysicalOptimizerRule for OptimizeProjections { plan: Arc, _config: &ConfigOptions, ) -> Result> { + println!("INITIAL"); print_plan(&plan); - // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); + let mut optimizer = ProjectionOptimizer::new_default(plan); // Insert the initial requirements to the root node, and run the rule. @@ -3592,7 +4586,8 @@ impl PhysicalOptimizerRule for OptimizeProjections { // optimized plan satisfies the initial schema order. optimized = optimized .map_data(|node| satisfy_initial_schema(node, initial_requirements))?; - + // println!("FINAL"); + // print_plan(&optimized.data.plan); Ok(optimized.data.plan) } @@ -3705,16 +4700,18 @@ fn analyze_requirements(node: &ProjectionOptimizer) -> ColumnRequirements { /// - The second element represents the column requirements for the right child. /// fn analyze_requirements_of_joins( - schema: SchemaRef, + left_child: &Arc, + right_child: &Arc, required_columns: &HashSet, left_size: usize, ) -> (ColumnRequirements, ColumnRequirements) { - let columns_in_schema = schema - .fields() - .iter() - .enumerate() - .map(|(idx, field)| Column::new(field.name(), idx)) - .collect::>(); + 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| { @@ -3725,17 +4722,14 @@ fn analyze_requirements_of_joins( } }) .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) } @@ -4167,6 +5161,14 @@ fn collect_column_indices(exprs: &[(Arc, String)]) -> Vec, String)]) -> Vec { + // Collect indices and remove duplicates. + exprs + .iter() + .map(|(expr, _)| expr.as_any().downcast_ref::().unwrap().index()) + .collect::>() +} + #[derive(Debug, PartialEq)] enum RewriteState { /// The expression is unchanged. @@ -4647,6 +5649,8 @@ fn rewrite_hash_join( right_input_plan: Arc, left_mapping: &HashMap, right_mapping: &HashMap, + mut projection: Option>, + new_mapping: HashMap, ) -> Result> { let new_on = update_join_on(hj.on(), left_mapping, right_mapping); let new_filter = hj.filter().map(|filter| { @@ -4681,18 +5685,52 @@ fn rewrite_hash_join( filter.schema().clone(), ) }); + let index_mapping = new_mapping + .into_iter() + .map(|(initial, target)| (initial.index(), target.index())) + .collect::>(); + projection = projection.map(|mut prj| { + prj.iter_mut().for_each(|idx| { + *idx = *index_mapping + .get(idx) + .clone() + .map(|index| index) + .unwrap_or(idx) + }); + prj + }); + + if HashJoinExec::try_new( + left_input_plan.clone(), + right_input_plan.clone(), + new_on.clone(), + new_filter.clone(), + hj.join_type(), + projection.clone(), + *hj.partition_mode(), + hj.null_equals_null(), + ) + .is_err() + { + print_plan(&left_input_plan); + print_plan(&right_input_plan); + println!("new_on :{:?}", new_on); + println!("new_filter :{:?}", new_filter); + println!(" hj.join_type(), :{:?}", hj.join_type(),); + println!("projection :{:?}", projection); + } + HashJoinExec::try_new( left_input_plan, right_input_plan, new_on, new_filter, hj.join_type(), - None, + projection, *hj.partition_mode(), hj.null_equals_null(), - )? - .with_projection(hj.projection.clone()) - .map(|hj| Arc::new(hj) as _) + ) + .map(|plan| Arc::new(plan) as _) } fn rewrite_nested_loop_join( @@ -5835,8 +6873,8 @@ mod tests { // 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]", + "ProjectionExec: expr=[c@0 as c_from_left, b@1 as b_from_left, a@2 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=[c@2, b@1, a@0, 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); @@ -5856,10 +6894,10 @@ mod tests { // 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 = [ - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c, c@4 as c]", - " 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", - " 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"]; + "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(()) diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index f17862032889..05e656b72f51 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2877,10 +2877,9 @@ SortExec: expr=[sn@2 ASC NULLS LAST] ----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)] ------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] --------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[sn@0 as sn, amount@3 as amount, zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency] -------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------------MemoryExec: partitions=1, partition_sizes=[1] +----------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] +------------MemoryExec: partitions=1, partition_sizes=[1] +------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -3873,12 +3872,11 @@ physical_plan ProjectionExec: expr=[LAST_VALUE(l.d)@1 as amount_usd] --AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 -------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] ---------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # reset partition number to 8. statement ok @@ -4030,14 +4028,13 @@ Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 physical_plan ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, c@3 as c, sum1@5 as sum1] -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)] ---------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -----------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true ---------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -----------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] +------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index da9b4168e7e0..135ab8075425 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -587,7 +587,7 @@ FROM t1 ---- 11 11 11 -# subsequent inner join +# subsequent inner join query III rowsort SELECT t1.t1_id, t2.t2_id, t3.t3_id FROM t1 diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 84c884b86716..61e95815f999 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -58,11 +58,10 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 ----CoalesceBatchesExec: target_batch_size=8192 -------ProjectionExec: expr=[a@1 as a] ---------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true # preserve_inner_join query IIII nosort @@ -102,13 +101,12 @@ GlobalLimitExec: skip=0, fetch=10 --SortPreservingMergeExec: [a2@0 ASC NULLS LAST,b@1 ASC NULLS LAST], fetch=10 ----ProjectionExec: expr=[a@0 as a2, b@1 as b] ------CoalesceBatchesExec: target_batch_size=8192 ---------ProjectionExec: expr=[a@0 as a, b@1 as b] -----------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true -------------CoalesceBatchesExec: target_batch_size=8192 ---------------FilterExec: d@3 = 3 -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +--------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)], projection=[a@0, b@1] +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true +----------CoalesceBatchesExec: target_batch_size=8192 +------------FilterExec: d@3 = 3 +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join query II nosort diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index b89647d919e8..cca23d5cff80 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1326,16 +1326,15 @@ Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] physical_plan AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t1_id@0 as t1_id] -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1354,16 +1353,15 @@ physical_plan ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] --AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] ----CoalesceBatchesExec: target_batch_size=2 -------ProjectionExec: expr=[t1_id@0 as t1_id] ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1387,16 +1385,15 @@ ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] --------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] ----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] ------------CoalesceBatchesExec: target_batch_size=2 ---------------ProjectionExec: expr=[t1_id@0 as t1_id] -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[1] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1456,15 +1453,14 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] ---------CoalescePartitionsExec -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1483,18 +1479,17 @@ Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_ physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int] -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 -------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 -------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1515,15 +1510,14 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] ---------CoalescePartitionsExec -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1542,18 +1536,17 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 -------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 -------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1575,12 +1568,11 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@1 as t1_id, t1_name@2 as t1_name] -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] ---------MemoryExec: partitions=1, partition_sizes=[1] ---------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1600,17 +1592,16 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@1 as t1_id, t1_name@2 as t1_name] -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 +----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 -------------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1632,14 +1623,13 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] ---------CoalescePartitionsExec -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +------CoalescePartitionsExec +--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1659,17 +1649,16 @@ Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t2_id@0 as t2_id, t1_id@2 as t1_id, t1_name@3 as t1_name] -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 -------------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 +----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 --------------MemoryExec: partitions=1, partition_sizes=[1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1689,12 +1678,11 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) --TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan CoalesceBatchesExec: target_batch_size=2 ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] -------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +----MemoryExec: partitions=1, partition_sizes=[1] +----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1712,17 +1700,16 @@ Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) --TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan CoalesceBatchesExec: target_batch_size=2 ---ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] +----CoalesceBatchesExec: target_batch_size=2 +------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 +--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -3423,12 +3410,11 @@ physical_plan ProjectionExec: expr=[LAST_VALUE(l.d)@1 as amount_usd] --AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 -------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] ---------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # run query above in multiple partitions statement ok diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 8a05e49192fd..c77e5fcb4045 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -636,21 +636,19 @@ Projection: lineitem.l_partkey ------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] physical_plan CoalesceBatchesExec: target_batch_size=8192 ---ProjectionExec: expr=[l_partkey@0 as l_partkey] -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15 -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------CoalesceBatchesExec: target_batch_size=8192 -------------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 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------CoalesceBatchesExec: target_batch_size=8192 -------------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true - +--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] +----CoalesceBatchesExec: target_batch_size=8192 +------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +--------CoalesceBatchesExec: target_batch_size=8192 +----------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 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true +----CoalesceBatchesExec: target_batch_size=8192 +------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +--------CoalesceBatchesExec: target_batch_size=8192 +----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -730,25 +728,23 @@ Aggregate: groupBy=[[part.p_partkey]], aggr=[[SUM(lineitem.l_extendedprice), AVG physical_plan AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] --CoalesceBatchesExec: target_batch_size=8192 -----ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] -------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true +----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] +------CoalesceBatchesExec: target_batch_size=8192 +--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +----------CoalesceBatchesExec: target_batch_size=8192 +------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true +----------CoalesceBatchesExec: target_batch_size=8192 +------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 --------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -----------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +----------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 +--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +------CoalesceBatchesExec: target_batch_size=8192 +--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index f118e9b4b0f8..53fee3a52e91 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -79,40 +79,35 @@ GlobalLimitExec: skip=0, fetch=10 ------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)] +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------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 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ---------------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 -----------------------------------------------------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 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------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 ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------------------FilterExec: l_returnflag@3 = R -------------------------------------------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 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - +--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +--------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 +----------------------------------------------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 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +------------------------------------FilterExec: l_returnflag@3 = R +--------------------------------------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 +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query ITRRTTTT select diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 3629bc151253..b666df91d973 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -84,53 +84,49 @@ GlobalLimitExec: skip=0, fetch=10 --------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 ---------------------------------------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, ps_supplycost], has_header=false -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] ---------------------------------FilterExec: n_name@1 = GERMANY +--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +----------------------------------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, ps_supplycost], has_header=false +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +------------------------------FilterExec: n_name@1 = GERMANY +--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] ------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] --------------CoalescePartitionsExec ----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] ---------------------------------FilterExec: n_name@1 = GERMANY +--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +----------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +------------------------------FilterExec: n_name@1 = GERMANY +--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query IR select diff --git a/datafusion/sqllogictest/test_files/tpch/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/q12.slt.part index a4eb58ec6752..8b3477cbd6f7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -67,18 +67,16 @@ SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] --------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[l_shipmode@1 as l_shipmode, o_orderpriority@3 as o_orderpriority] -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] -----------------------------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 -------------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------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_orderpriority], has_header=false - +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] +--------------------------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 +----------------------------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 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +----------------------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_orderpriority], has_header=false query TII diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index 19aeaa5894aa..9de8cce1ab47 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -65,18 +65,17 @@ GlobalLimitExec: skip=0, fetch=10 ----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] ------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] -------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -----------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -------------------------------------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 +----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +--------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% +----------------------------------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 155005210d23..ee48940caa95 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -46,18 +46,17 @@ ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] --------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_type@4 as p_type] -------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 -------------------------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 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +--------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 +----------------------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 +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 6bc9c5f87760..71f869dd0b33 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -74,40 +74,38 @@ physical_plan SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] --SortExec: expr=[s_suppkey@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=8192 -------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue] ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue] -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)] +------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] +--------CoalesceBatchesExec: target_batch_size=8192 +----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 +------------CoalesceBatchesExec: target_batch_size=8192 +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false ---------------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -------------------------------------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 -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 ---------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] -----------------CoalescePartitionsExec -------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] ---------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -------------------------------------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 +----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +--------------------------------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 +--------CoalesceBatchesExec: target_batch_size=8192 +----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +--------------CoalescePartitionsExec +----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +--------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +--------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +----------------------------------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 query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 080a3592a5d6..8a14d77878f9 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -83,17 +83,16 @@ GlobalLimitExec: skip=0, fetch=10 ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -------------------------------------------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], has_header=false ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------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 +----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +----------------------------------------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], has_header=false +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) +--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------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 ----------------------------CoalesceBatchesExec: target_batch_size=8192 ------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index 600e7e5fa78b..bb4ed35af486 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -56,27 +56,25 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] --------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] -------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] +----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +------------CoalesceBatchesExec: target_batch_size=8192 +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +--------------------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 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------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 ---------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] -----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -------------------------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], has_header=false +----------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------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 +------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] +--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +----------------------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], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index 5cacf104ee06..ed639b71bc3d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -76,23 +76,21 @@ SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ------------CoalesceBatchesExec: target_batch_size=8192 --------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------------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_totalprice, o_orderdate], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------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 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +--------------------------------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_totalprice, o_orderdate], has_header=false +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +------------------------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 ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] --------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 8b3f51eb25a7..89e5e53ded72 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -69,20 +69,19 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] --------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -------------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 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------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] -----------------------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 -------------------------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 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 -----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +----------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] +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------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] +--------------------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 +----------------------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 +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 +--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false query R diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index aa33680eedf7..6547f93eb069 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -16,180 +16,6 @@ # specific language governing permissions and limitations # under the License. -query TT -explain select - s_acctbal, - s_name, - n_name, - p_partkey, - p_mfgr, - s_address, - s_phone, - s_comment -from - part, - supplier, - partsupp, - nation, - region -where - p_partkey = ps_partkey - and s_suppkey = ps_suppkey - and p_size = 15 - and p_type like '%BRASS' - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'EUROPE' - and ps_supplycost = ( - select - min(ps_supplycost) - from - partsupp, - supplier, - nation, - region - where - p_partkey = ps_partkey - and s_suppkey = ps_suppkey - and s_nationkey = n_nationkey - and n_regionkey = r_regionkey - and r_name = 'EUROPE' -) -order by - s_acctbal desc, - n_name, - s_name, - p_partkey -limit 10; ----- -logical_plan -Limit: skip=0, fetch=10 ---Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 -----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment -------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) ---------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name -----------Inner Join: nation.n_regionkey = region.r_regionkey -------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey ---------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost -------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey ---------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost -----------------------Inner Join: part.p_partkey = partsupp.ps_partkey -------------------------Projection: part.p_partkey, part.p_mfgr ---------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") -----------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8("%BRASS")] -------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] ---------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] -----------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -------------Projection: region.r_regionkey ---------------Filter: region.r_name = Utf8("EUROPE") -----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey -------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] ---------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost -----------------Inner Join: nation.n_regionkey = region.r_regionkey -------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey ---------------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey -------------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey ---------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] ---------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------TableScan: nation projection=[n_nationkey, n_regionkey] -------------------Projection: region.r_regionkey ---------------------Filter: region.r_name = Utf8("EUROPE") -----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] -physical_plan -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] ---GlobalLimitExec: skip=0, fetch=10 -----SortPreservingMergeExec: [s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST], fetch=10 -------SortExec: TopK(fetch=10), expr=[s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, n_name@8 as n_name] -------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0), (ps_supplycost@7, MIN(partsupp.ps_supplycost)@1)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, ps_supplycost@7 as ps_supplycost, n_name@8 as n_name] -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------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_phone@6 as s_phone, s_acctbal@7 as s_acctbal, s_comment@8 as s_comment, ps_supplycost@2 as ps_supplycost, n_name@10 as n_name, n_regionkey@11 as n_regionkey] ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@5, n_nationkey@0)] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_supplycost@3 as ps_supplycost, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment] -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] -----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)] -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] ---------------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------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 -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------------------------------------------------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 ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------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 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------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 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ---------------------------------FilterExec: r_name@1 = EUROPE -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 -------------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -------------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)] ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 ---------------------------------------------------------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 -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -----------------------------------------FilterExec: r_name@1 = EUROPE -------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - query RTTITTTT select diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 17ff7ff08d99..35055c4b9c30 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -292,21 +292,20 @@ UnionExec --------------MemoryExec: partitions=1, partition_sizes=[1] --ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] ----CoalesceBatchesExec: target_batch_size=2 -------ProjectionExec: expr=[id@0 as id, name@1 as name] ---------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 ---------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -----------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------MemoryExec: partitions=1, partition_sizes=[1] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +----------------CoalesceBatchesExec: target_batch_size=2 +------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[1] +--------CoalesceBatchesExec: target_batch_size=2 +----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort From 5b4e107791e58610e0f9e7f24dd8bc4a16149244 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 19 Mar 2024 13:22:51 +0300 Subject: [PATCH 37/85] tests pass --- .../optimize_projections.rs | 515 ++++-------------- .../sqllogictest/test_files/subquery.slt | 86 ++- .../sqllogictest/test_files/tpch/q20.slt.part | 79 ++- .../sqllogictest/test_files/tpch/q21.slt.part | 93 ++-- .../sqllogictest/test_files/tpch/q22.slt.part | 21 +- .../sqllogictest/test_files/tpch/q3.slt.part | 45 +- .../sqllogictest/test_files/tpch/q4.slt.part | 27 +- .../sqllogictest/test_files/tpch/q5.slt.part | 85 ++- .../sqllogictest/test_files/tpch/q7.slt.part | 83 ++- .../sqllogictest/test_files/tpch/q8.slt.part | 120 ++-- .../sqllogictest/test_files/tpch/q9.slt.part | 78 ++- 11 files changed, 437 insertions(+), 795 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 64bd857f2f36..277fca94bad6 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -79,7 +79,6 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::displayable; use datafusion_physical_plan::insert::FileSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, @@ -97,13 +96,6 @@ use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use itertools::Itertools; -fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) -} - /// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary /// fields for column requirements and changed indices of columns. #[derive(Debug, Clone)] @@ -139,9 +131,6 @@ impl ProjectionOptimizer { /// 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 { - // println!("1)NODE"); - // print_plan(&self.plan); - // println!("reqs: {:?}", self.required_columns); // 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 @@ -150,19 +139,14 @@ impl ProjectionOptimizer { return Ok(self); } - let x: ProjectionOptimizer = if self.plan.as_any().is::() { + 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() - }?; - // println!("2)NODE"); - // print_plan(&x.plan); - // println!("reqs: {:?}", x.children_nodes[0].required_columns); - // println!("maps: {:?}", x.schema_mapping); - Ok(x) + } } /// The function tries 4 cases: @@ -211,7 +195,7 @@ impl ProjectionOptimizer { 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() + return join.data.adjust_node_with_requirements(); } projection => Ok(projection.data), }; @@ -240,7 +224,7 @@ impl ProjectionOptimizer { Ok(self) } - /// If it is beneficial. unifies the projection and its input, which is also a [`ProjectionExec`]. + /// 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 { @@ -253,6 +237,7 @@ impl ProjectionOptimizer { 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)); @@ -390,7 +375,7 @@ impl ProjectionOptimizer { } let new_hash_join = - Arc::new(hash_join.with_projection(Some(projection_index.to_vec()))?) + Arc::new(hash_join.with_projection(Some(projection_index.clone()))?) as Arc; // Build projection expressions for update_expr. Zip the projection_index with the new_hash_join output schema fields. @@ -1247,6 +1232,7 @@ impl ProjectionOptimizer { .flat_map(|(_, right_on)| collect_columns(right_on)) .collect::>(), ); + hj_right_requirements.extend( hj.filter() .map(|filter| { @@ -1309,6 +1295,7 @@ impl 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 .iter() .map(|req| { @@ -1316,6 +1303,7 @@ impl ProjectionOptimizer { (Arc::new(req.clone()) as Arc, name) }) .collect::>(); + let new_right_projection = ProjectionExec::try_new( right_projection_exprs, hj.right().clone(), @@ -1412,25 +1400,6 @@ impl ProjectionOptimizer { }) .collect() }); - if HashJoinExec::try_new( - new_left_node.plan.clone(), - new_right_node.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - new_projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_node.plan); - print_plan(&new_right_node.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", new_projection); - } let new_hash_join = HashJoinExec::try_new( new_left_node.plan.clone(), @@ -1484,7 +1453,7 @@ impl ProjectionOptimizer { .map(|(idx, col)| (col.clone(), Column::new(col.name(), idx))) .collect::>(); let mut right_mapping = - hj_right_requirements.into_iter().collect_vec(); + hj_right_requirements.iter().cloned().collect_vec(); right_mapping.sort_by_key(|col| col.index()); let right_mapping = right_mapping .into_iter() @@ -1556,28 +1525,10 @@ impl ProjectionOptimizer { }) .collect() }); - if HashJoinExec::try_new( - new_left_node.plan.clone(), - hj.right().clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - new_projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_node.plan); - print_plan(&hj.right()); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", new_projection); - } + let new_hash_join = HashJoinExec::try_new( new_left_node.plan.clone(), - hj.right().clone(), + self.children_nodes[1].plan.clone(), new_on, new_filter, hj.join_type(), @@ -1585,20 +1536,20 @@ impl ProjectionOptimizer { *hj.partition_mode(), hj.null_equals_null(), )?; + let mut right_ = self.children_nodes[1].clone(); + right_.required_columns = hj_right_requirements; return Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), required_columns: HashSet::new(), schema_mapping: HashMap::new(), - children_nodes: vec![ - new_left_node, - self.children_nodes[1].clone(), - ], + children_nodes: vec![new_left_node, right_], }); } (true, false) => { 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 .iter() .map(|req| { @@ -1606,6 +1557,7 @@ impl ProjectionOptimizer { (Arc::new(req.clone()) as Arc, name) }) .collect::>(); + let new_right_projection = ProjectionExec::try_new( right_projection_exprs, hj.right().clone(), @@ -1702,27 +1654,9 @@ impl ProjectionOptimizer { }) .collect() }); - if HashJoinExec::try_new( - hj.left().clone(), - new_right_node.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - new_projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&hj.left()); - print_plan(&new_right_node.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", new_projection); - } + let new_hash_join = HashJoinExec::try_new( - hj.left().clone(), + self.children_nodes[0].plan.clone(), new_right_node.plan.clone(), new_on, new_filter, @@ -1731,14 +1665,13 @@ impl ProjectionOptimizer { *hj.partition_mode(), hj.null_equals_null(), )?; + let mut left_ = self.children_nodes[0].clone(); + left_.required_columns = hj_left_requirements; return Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), required_columns: HashSet::new(), schema_mapping: HashMap::new(), - children_nodes: vec![ - self.children_nodes[0].clone(), - new_right_node, - ], + children_nodes: vec![left_, new_right_node], }); } (true, true) => { @@ -1965,25 +1898,7 @@ impl ProjectionOptimizer { }) .collect() }); - if HashJoinExec::try_new( - new_left_node.plan.clone(), - new_right_node.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - new_projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_node.plan); - print_plan(&new_right_node.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", new_projection); - } + let new_hash_join = HashJoinExec::try_new( new_left_node.plan.clone(), new_right_node.plan.clone(), @@ -2108,25 +2023,7 @@ impl ProjectionOptimizer { }) .collect() }); - if HashJoinExec::try_new( - new_left_node.plan.clone(), - hj.right().clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - new_projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_node.plan); - print_plan(&hj.right()); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", new_projection); - } + let new_hash_join = HashJoinExec::try_new( new_left_node.plan.clone(), hj.right().clone(), @@ -2254,25 +2151,7 @@ impl ProjectionOptimizer { }) .collect() }); - if HashJoinExec::try_new( - hj.left().clone(), - new_right_node.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - new_projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&hj.left()); - print_plan(&new_right_node.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", new_projection); - } + let new_hash_join = HashJoinExec::try_new( hj.left().clone(), new_right_node.plan.clone(), @@ -2351,25 +2230,7 @@ impl ProjectionOptimizer { analyzed_join_left, analyzed_join_right, )?; - if HashJoinExec::try_new( - new_left_child.plan.clone(), - new_right_child.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - None, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_child.plan); - print_plan(&new_right_child.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :None"); - } + let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), new_right_child.plan.clone(), @@ -2408,25 +2269,7 @@ impl ProjectionOptimizer { right_child.required_columns = update_right_child_requirements(&required_columns, left_size); - if HashJoinExec::try_new( - new_left_child.plan.clone(), - right_child.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - None, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_child.plan); - print_plan(&right_child.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :None",); - } + let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -2472,25 +2315,7 @@ impl ProjectionOptimizer { left_child.required_columns = collect_left_used_columns(required_columns, left_size); - if HashJoinExec::try_new( - left_child.plan.clone(), - new_right_child.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - None, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&left_child.plan); - print_plan(&new_right_child.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :None",); - } + let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -2535,25 +2360,7 @@ impl ProjectionOptimizer { let (new_left_child, left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; - if HashJoinExec::try_new( - new_left_child.plan.clone(), - right_child.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - None, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&new_left_child.plan); - print_plan(&right_child.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :None",); - } + let plan = Arc::new(HashJoinExec::try_new( new_left_child.plan.clone(), right_child.plan.clone(), @@ -2606,25 +2413,7 @@ impl ProjectionOptimizer { ); let (new_right_child, right_schema_mapping) = self .insert_projection_below_right_child(analyzed_join_right)?; - if HashJoinExec::try_new( - left_child.plan.clone(), - new_right_child.plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - None, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&left_child.plan); - print_plan(&new_right_child.plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :None",); - } + let plan = Arc::new(HashJoinExec::try_new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -4194,38 +3983,73 @@ impl ProjectionOptimizer { )?; update_mapping(&mut self, all_mappings) } else if let Some(hj) = plan_any.downcast_ref::() { - let projection = hj.projection.clone(); - let left_input_size = self.children_nodes[0].plan.schema().fields().len(); - let left_size = projection - .clone() - .unwrap_or((0..hj.schema().fields().len()).collect()) - .iter() - .filter(|idx| **idx < left_input_size) - .count(); 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)| { + let projection = hj.projection.clone(); + 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(), + ) + }); + let index_mapping = left_mapping + .into_iter() + .map(|(col1, col2)| (col1.index(), col2.index())) + .chain(right_mapping.into_iter().map(|(col1, col2)| { ( - Column::new(initial.name(), initial.index() + left_size), - Column::new(new.name(), new.index() + left_size), + col1.index() + hj.children()[0].schema().fields().len(), + col2.index() + + self.children_nodes[0].plan.schema().fields().len(), ) })) .collect::>(); - - self.plan = rewrite_hash_join( - hj, + let new_projection = projection.map(|mut prj| { + prj.iter_mut() + .for_each(|idx| *idx = *index_mapping.get(idx).unwrap_or(idx)); + prj + }); + self.plan = HashJoinExec::try_new( self.children_nodes[0].plan.clone(), self.children_nodes[1].plan.clone(), - &left_mapping, - &right_mapping, - projection, - new_mapping, - )?; - - // self.schema_mapping = empty; + new_on, + new_filter, + hj.join_type(), + new_projection, + *hj.partition_mode(), + hj.null_equals_null(), + ) + .map(|plan| Arc::new(plan) as _)?; self.schema_mapping = HashMap::new(); } else if let Some(nlj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); @@ -4436,12 +4260,14 @@ impl ProjectionOptimizer { unreachable!() } } else { - self.plan = self.plan.with_new_children( + 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)) @@ -4528,12 +4354,6 @@ impl ConcreteTreeNode for ProjectionOptimizer { fn with_new_children(mut self, children: Vec) -> Result { self.children_nodes = children; - // println!("INDEX UPDATE"); - // print_plan(&self.plan); - // self.children_nodes.iter().for_each(|c| { - // print_plan(&c.plan); - // println!("MAP: {:?}", c.schema_mapping); - // }); self = match self.index_updater()? { new_node if new_node.transformed => new_node.data, @@ -4568,8 +4388,6 @@ impl PhysicalOptimizerRule for OptimizeProjections { plan: Arc, _config: &ConfigOptions, ) -> Result> { - println!("INITIAL"); - print_plan(&plan); // Collect initial columns requirements from the plan's schema. let initial_requirements = collect_columns_in_plan_schema(&plan); @@ -4586,8 +4404,7 @@ impl PhysicalOptimizerRule for OptimizeProjections { // optimized plan satisfies the initial schema order. optimized = optimized .map_data(|node| satisfy_initial_schema(node, initial_requirements))?; - // println!("FINAL"); - // print_plan(&optimized.data.plan); + Ok(optimized.data.plan) } @@ -5147,25 +4964,12 @@ fn collect_columns_in_join_conditions( .collect::>() } -/// Collect all column indices from the given projection expressions in the index order. -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 -} - fn collect_column_indices_hj(exprs: &[(Arc, String)]) -> Vec { // Collect indices and remove duplicates. exprs .iter() - .map(|(expr, _)| expr.as_any().downcast_ref::().unwrap().index()) + .flat_map(|(expr, _)| collect_columns(expr)) + .map(|col| col.index()) .collect::>() } @@ -5185,9 +4989,9 @@ enum RewriteState { /// 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`. +/// 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`: /// @@ -5628,111 +5432,6 @@ fn rewrite_sort_preserving_merge( ) as _) } -/// Rewrites a hash join execution plan with updated column indices. -/// -/// Updates the join conditions and filter expressions in a hash join plan based on provided column index mappings -/// for both left and right input plans. -/// -/// # Arguments -/// * `hj` - The original `HashJoinExec` plan. -/// * `left_input_plan` - The left input execution plan. -/// * `right_input_plan` - The right input execution plan. -/// * `left_mapping` - A hashmap with old and new column index mappings for the left input. -/// * `right_mapping` - A hashmap with old and new column index mappings for the right input. -/// * `left_size` - The size of the left input columns set. -/// -/// # Returns -/// A `Result` containing the new `HashJoinExec` wrapped in an `Arc`. -fn rewrite_hash_join( - hj: &HashJoinExec, - left_input_plan: Arc, - right_input_plan: Arc, - left_mapping: &HashMap, - right_mapping: &HashMap, - mut projection: Option>, - new_mapping: HashMap, -) -> Result> { - 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(), - ) - }); - let index_mapping = new_mapping - .into_iter() - .map(|(initial, target)| (initial.index(), target.index())) - .collect::>(); - projection = projection.map(|mut prj| { - prj.iter_mut().for_each(|idx| { - *idx = *index_mapping - .get(idx) - .clone() - .map(|index| index) - .unwrap_or(idx) - }); - prj - }); - - if HashJoinExec::try_new( - left_input_plan.clone(), - right_input_plan.clone(), - new_on.clone(), - new_filter.clone(), - hj.join_type(), - projection.clone(), - *hj.partition_mode(), - hj.null_equals_null(), - ) - .is_err() - { - print_plan(&left_input_plan); - print_plan(&right_input_plan); - println!("new_on :{:?}", new_on); - println!("new_filter :{:?}", new_filter); - println!(" hj.join_type(), :{:?}", hj.join_type(),); - println!("projection :{:?}", projection); - } - - HashJoinExec::try_new( - left_input_plan, - right_input_plan, - new_on, - new_filter, - hj.join_type(), - projection, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .map(|plan| Arc::new(plan) as _) -} - fn rewrite_nested_loop_join( nlj: &NestedLoopJoinExec, left_input_plan: Arc, @@ -6416,22 +6115,6 @@ mod tests { 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_csv_after_projection() -> Result<()> { let csv = create_projecting_csv_exec(); diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 89b8595d4d25..66ac669c73c7 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -189,16 +189,15 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t1_id@2 as t1_id] -------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)] ---------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t1_id@2] +------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] --------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +--------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------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 @@ -223,17 +222,16 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int * Float64(1)) + Int64(1) AS t2 physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as SUM(t2.t2_int * Float64(1)) + Int64(1), t1_id@2 as t1_id] -------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] ---------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] -----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] -------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----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] +------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] +--------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +----------CoalesceBatchesExec: target_batch_size=2 +------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int * Float64(1))] +----------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query IR rowsort SELECT t1_id, (SELECT sum(t2_int * 1.0) + 1 FROM t2 WHERE t2.t2_id = t1.t1_id) as t2_sum from t1 @@ -258,19 +256,18 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum physical_plan ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as SUM(t2.t2_int)] -------HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 -------------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] ---------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) -----------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)], projection=[t1_id@0, SUM(t2.t2_int)@1] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t2_id@1], 4), input_partitions=4 +----------ProjectionExec: expr=[SUM(t2.t2_int)@2 as SUM(t2.t2_int), t2_id@0 as t2_id] +------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id, Utf8("a")@1 as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) +--------------CoalesceBatchesExec: target_batch_size=2 +----------------RepartitionExec: partitioning=Hash([t2_id@0, Utf8("a")@1], 4), input_partitions=4 +------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id, a as Utf8("a")], aggr=[SUM(t2.t2_int)], ordering_mode=PartiallySorted([1]) +--------------------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 group by t2_id, 'a') as t2_sum from t1 @@ -296,18 +293,17 @@ Projection: t1.t1_id, __scalar_sq_1.SUM(t2.t2_int) AS t2_sum physical_plan ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] --CoalesceBatchesExec: target_batch_size=2 -----ProjectionExec: expr=[SUM(t2.t2_int)@1 as SUM(t2.t2_int), t1_id@2 as t1_id] -------HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------FilterExec: SUM(t2.t2_int)@1 < 3 -------------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------CoalesceBatchesExec: target_batch_size=2 -----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t1_id@2] +------CoalesceBatchesExec: target_batch_size=2 +--------FilterExec: SUM(t2.t2_int)@1 < 3 +----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +------------CoalesceBatchesExec: target_batch_size=2 +--------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 +----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] +------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +------CoalesceBatchesExec: target_batch_size=2 +--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 +----------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/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index 85fcbf12b01d..d6f2da9137c0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -84,52 +84,49 @@ physical_plan SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] --SortExec: expr=[s_name@0 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=8192 -------ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] ---------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] +------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] +--------CoalesceBatchesExec: target_batch_size=8192 +----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +------------CoalesceBatchesExec: target_batch_size=8192 +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------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 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------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 +----------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +------------------------FilterExec: n_name@1 = CANADA +--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +--------CoalesceBatchesExec: target_batch_size=8192 +----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +------------CoalesceBatchesExec: target_batch_size=8192 +--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] ------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -----------------------------FilterExec: n_name@1 = CANADA -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +----------------------------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 ------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 ---------------------------------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 +--------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------------FilterExec: p_name@1 LIKE forest% ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false ---------------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -----------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -----------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -----------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 -------------------------------------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 +------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +--------------------------------FilterExec: p_name@1 LIKE forest% +----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] +------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 +--------------------------------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 query TT select diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index 57f548fd885f..a7a5230b310f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -98,58 +98,53 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] --------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[s_name@0 as s_name] -------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)] +----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 ----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 +------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] --------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] -----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------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 -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] ---------------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -----------------------------------------------------------------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 ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] -----------------------------------------------------FilterExec: o_orderstatus@1 = F -------------------------------------------------------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 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------------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 +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +--------------------------------------------------------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 --------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------------------------FilterExec: n_name@1 = SAUDI ARABIA ---------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -------------------------------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 - +----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] +----------------------------------------------FilterExec: o_orderstatus@1 = F +------------------------------------------------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 +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +--------------------------------------FilterExec: n_name@1 = SAUDI ARABIA +----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +--------------------------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 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +--------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +----------------------------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 query TI select diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index 3a650c2f3574..3f7dcd1be990 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -84,17 +84,16 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] --------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] ----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] -----------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------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") }]) ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------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 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -----------------------------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 +--------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------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") }]) +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------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 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +--------------------------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 ------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] --------------------CoalescePartitionsExec ----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index 939dc4d30ef2..93b1737052f6 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -68,32 +68,29 @@ GlobalLimitExec: skip=0, fetch=10 ------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 --------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -----------------------------------------FilterExec: c_mktsegment@1 = BUILDING -------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +----------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] +------------------------------------FilterExec: c_mktsegment@1 = BUILDING +--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------FilterExec: o_orderdate@2 < 9204 -----------------------------------------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 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -------------------------------FilterExec: l_shipdate@3 > 9204 ---------------------------------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 - +----------------------------------FilterExec: o_orderdate@2 < 9204 +------------------------------------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 +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +----------------------------FilterExec: l_shipdate@3 > 9204 +------------------------------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 query IRDI select diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index 3f229670428a..2776ead9dc7f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -61,20 +61,19 @@ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] --------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] -------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] -----------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 -------------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -----------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 -------------------------------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 +----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] +--------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 +----------------------------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 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] +--------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 +----------------------------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 query TI select diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index cd0f5124da93..c6dd9f54b331 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -75,56 +75,51 @@ SortPreservingMergeExec: [revenue@1 DESC] ----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 ------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] --------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 ------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] ----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] ---------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)] -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] -------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)] +------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] -----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ---------------------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 -----------------------------------------------------------------------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 +----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false --------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------------------------------------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_extendedprice, l_discount], has_header=false -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 ---------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -----------------------------FilterExec: r_name@1 = ASIA +----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +----------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 +------------------------------------------------------------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 +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +----------------------------------------------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_extendedprice, l_discount], has_header=false +----------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +--------------------------------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 +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +--------------------------FilterExec: r_name@1 = ASIA +----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false query TR select diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 7f912f36aeb4..70c9fd63771e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -93,56 +93,51 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L ------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] --------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, l_shipdate@2 as l_shipdate, n_name@4 as n_name, n_name@6 as n_name] ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@6 as n_name] -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)] +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 +----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)] +--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 +----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] -------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] ---------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 -----------------------------------------------------------------------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_extendedprice, l_discount, l_shipdate], has_header=false +------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ---------------------------------------------------------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], has_header=false -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 +------------------------------------------------------------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_extendedprice, l_discount, l_shipdate], has_header=false +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +------------------------------------------------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], has_header=false ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query TTRR select diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 7d7f94a183e0..02016a444355 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -99,77 +99,69 @@ SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] ------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] --------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] ----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@2 as o_orderdate, n_name@4 as n_name] ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] -------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 --------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] ------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)] +--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 +----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 --------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] -------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] ---------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] -----------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)] +----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] +------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------------------------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] +--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -----------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] ---------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] -----------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL ---------------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false -----------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 ---------------------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 -----------------------------------------------------------------------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 +--------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL +------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -------------------------------FilterExec: r_name@1 = AMERICA +------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 +------------------------------------------------------------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 +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 --------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false - +----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +----------------------------FilterExec: r_name@1 = AMERICA +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false query RR select diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 8da8131a24d1..48e7b236ac4a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -86,53 +86,47 @@ GlobalLimitExec: skip=0, fetch=10 --------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] ----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] ------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------ProjectionExec: expr=[l_quantity@0 as l_quantity, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, ps_supplycost@4 as ps_supplycost, o_orderdate@5 as o_orderdate, n_name@7 as n_name] -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] +--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] --------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] -----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)] +----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 +------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------------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] ------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +--------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] ---------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------------------------------------------------FilterExec: p_name@1 LIKE %green% ---------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 ---------------------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +--------------------------------------------------------------FilterExec: p_name@1 LIKE %green% +----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false ------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -------------------------------------------------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 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ---------------------------------------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], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - +--------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +----------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +------------------------------------------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 +------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +----------------------------------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], has_header=false +----------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query TRR select From e384bb13f9bfc398534400a2639d9a47b03aaabf Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 19 Mar 2024 14:07:54 +0300 Subject: [PATCH 38/85] get upstream changes --- datafusion/core/src/dataframe/mod.rs | 1 - .../src/equivalence/projection.rs | 1 - datafusion/physical-plan/src/projection.rs | 1 - datafusion/sqllogictest/test_files/window.slt | 220 +++++++++--------- 4 files changed, 108 insertions(+), 115 deletions(-) diff --git a/datafusion/core/src/dataframe/mod.rs b/datafusion/core/src/dataframe/mod.rs index d0da1b0c86b3..eea5fc1127ce 100644 --- a/datafusion/core/src/dataframe/mod.rs +++ b/datafusion/core/src/dataframe/mod.rs @@ -2911,7 +2911,6 @@ mod tests { &["c2_c1", "c2_c2"], None, )?; - println!("join_type: {:?}", join_type); let physical_plan = join.create_physical_plan().await?; let out_partitioning = physical_plan.output_partitioning(); let join_schema = physical_plan.schema(); diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 7bc82587f5de..732310f9ee2f 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -51,7 +51,6 @@ impl ProjectionMapping { expr: Vec<(Arc, String)>, input_schema: &SchemaRef, ) -> Result { - // Construct a map from the input expressions to the output expression of the projection: expr.into_iter() .enumerate() .map(|(expr_idx, (expression, name))| { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index b8a603a71e50..eaea84b2c327 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -73,7 +73,6 @@ impl ProjectionExec { .iter() .map(|(_, alias)| alias.to_string()) .collect::>(); - // construct a map from the input expressions to the output expression of the Projection let projection_mapping = ProjectionMapping::try_new(expr, &input_schema)?; expr = projection_mapping diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 15ec5ebda790..9fadc7f9ca84 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -359,7 +359,7 @@ SortPreservingMergeExec: [b@0 ASC NULLS LAST] --ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] ----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted ------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] ------------CoalesceBatchesExec: target_batch_size=8192 --------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 @@ -1209,9 +1209,9 @@ Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregat --------TableScan: aggregate_test_100 projection=[c8, c9] physical_plan ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum2] ---BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 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(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true @@ -1230,9 +1230,9 @@ Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregat --------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)) }] -----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [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(Int8(NULL)), end_bound: CurrentRow }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [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(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true @@ -1254,10 +1254,10 @@ Sort: aggregate_test_100.c2 ASC NULLS LAST physical_plan SortExec: expr=[c2@0 ASC NULLS LAST] --ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)) }] -------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, 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(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true @@ -1279,19 +1279,18 @@ Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggr --------TableScan: aggregate_test_100 projection=[c1, c2, c4] physical_plan ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] ---BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=4096 --------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] --------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=4096 ------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 --------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true - # test_window_agg_sort_reversed_plan # Only 1 SortExec was added query TT @@ -1311,8 +1310,8 @@ Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregat physical_plan ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as sum2] --GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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)) }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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)) }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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] --------SortExec: expr=[c9@0 DESC] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -1352,8 +1351,8 @@ Projection: aggregate_test_100.c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [ physical_plan ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] --GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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)) }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)) }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)) }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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)) }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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 }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)), is_causal: false }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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 }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -1395,13 +1394,12 @@ Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 physical_plan ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2] --GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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)) }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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] ------SortExec: expr=[c9@0 ASC NULLS LAST] ---------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)) }], mode=[Sorted] +--------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] ----------SortExec: expr=[c9@0 DESC] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true - query III SELECT c9, @@ -1438,10 +1436,10 @@ physical_plan ProjectionExec: expr=[c9@0 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@3 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@1 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn2] --GlobalLimitExec: skip=0, fetch=5 ----ProjectionExec: expr=[c9@2 as c9, 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 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, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, 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 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] -------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)) }], mode=[Sorted] +------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] --------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -----------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)) }], mode=[Sorted] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }], mode=[Sorted] +----------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] +------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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] --------------SortExec: expr=[c9@2 DESC,c1@0 DESC] ----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true @@ -1521,19 +1519,19 @@ physical_plan 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] --GlobalLimitExec: skip=0, fetch=5 ----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] -------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)) }, 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 }, 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)) }, 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)) }] +------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 }] --------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] -----------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 }], mode=[Sorted] +----------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] ------------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST] ---------------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 }], mode=[Sorted] +--------------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] ----------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC] -------------------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 }], mode=[Sorted] +------------------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] --------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC] -----------------------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)) }, 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)) }, 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)) }, 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 }] -------------------------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)) }, 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 }, 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)) }, 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)) }] +----------------------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 }] +------------------------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 }] --------------------------SortExec: expr=[c3@2 DESC NULLS LAST] -----------------------------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)) }, 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 }, 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)) }, 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)) }] -------------------------------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 }], mode=[Sorted] +----------------------------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 }] +------------------------------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] --------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST] ----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true @@ -1608,8 +1606,8 @@ physical_plan ProjectionExec: expr=[c9@0 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@1 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] --GlobalLimitExec: skip=0, fetch=5 ----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 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, 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 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] -------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)) }], mode=[Sorted] ---------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)) }], mode=[Sorted] +------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] +--------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] ----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true @@ -1652,8 +1650,8 @@ physical_plan ProjectionExec: expr=[c9@0 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@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@1 as sum2] --GlobalLimitExec: skip=0, fetch=5 ----ProjectionExec: expr=[c9@1 as c9, 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 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, 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 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] -------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)) }], mode=[Sorted] ---------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)) }], mode=[Sorted] +------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] +--------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] ----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true @@ -1699,9 +1697,9 @@ physical_plan ProjectionExec: expr=[c3@0 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@1 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@2 as sum2] --GlobalLimitExec: skip=0, fetch=5 ----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 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, 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 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] -------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)) }] +------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 }] --------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] -----------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 }], mode=[Sorted] +----------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] ------------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] --------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] ----------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] @@ -1797,18 +1795,17 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 ----ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC] ----------CoalesceBatchesExec: target_batch_size=4096 ------------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 --------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 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 }], mode=[Sorted] +------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 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] --------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] ----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true - query III SELECT c3, SUM(c9) OVER(ORDER BY c3 DESC, c9 DESC, c2 ASC) as sum1, @@ -1839,7 +1836,7 @@ Sort: aggregate_test_100.c1 ASC NULLS LAST physical_plan SortPreservingMergeExec: [c1@0 ASC NULLS LAST] --ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4096 ----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 @@ -1968,7 +1965,7 @@ Sort: aggregate_test_100.c1 ASC NULLS LAST physical_plan SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] --ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4096 ----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 @@ -1994,10 +1991,10 @@ Sort: aggregate_test_100.c1 ASC NULLS LAST physical_plan SortExec: expr=[c1@0 ASC NULLS LAST] --ProjectionExec: expr=[c1@0 as c1, 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 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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)) }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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] ------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] --------SortExec: expr=[c9@1 ASC NULLS LAST] -----------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 3 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 3 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(3)) }], mode=[Sorted] +----------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 3 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 3 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(3)), is_causal: false }], mode=[Sorted] ------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] --------------CoalesceBatchesExec: target_batch_size=4096 ----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 @@ -2086,16 +2083,15 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[c9@0 ASC NULLS LAST] ----ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, 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.c2, 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(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, 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.c2, 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(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] --------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c9@3 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] -----------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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)) }] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] 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, aggregate_test_100.c2] 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(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] ---------------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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)) }] +----------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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 }] +------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] 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, aggregate_test_100.c2] 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(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +--------------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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 }] ----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] ------------------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, @@ -2143,12 +2139,12 @@ physical_plan ProjectionExec: expr=[c9@0 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@2 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@4 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@1 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@3 as sum4] --GlobalLimitExec: skip=0, fetch=5 ----ProjectionExec: expr=[c9@1 as c9, 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 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@4 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@5 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, 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 SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] -------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)) }], mode=[Sorted] +------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] --------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] -----------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)) }] +----------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 }] ------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] ---------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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)) }], mode=[Sorted] -----------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }] +--------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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] +----------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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 }] ------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] --------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] ----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true @@ -2191,12 +2187,13 @@ GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] ----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] ------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] ---------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)) }], mode=[Sorted] +--------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] ----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, 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@4 as 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] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }], mode=[Sorted] +------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] --------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] ----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true + query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, SUM(c12) OVER(ORDER BY c1 GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING) as sum2 @@ -2229,7 +2226,7 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------SortExec: expr=[c9@0 ASC NULLS LAST] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2268,7 +2265,7 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------SortExec: expr=[c9@0 DESC] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2308,7 +2305,7 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[rn1@1 DESC] ----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2351,7 +2348,7 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] ----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2404,7 +2401,7 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------SortExec: expr=[c9@0 DESC] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2428,7 +2425,7 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[c5@0 as c5, c9@1 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------SortExec: expr=[CAST(c9@1 AS Int32) + c5@0 DESC] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], has_header=true @@ -2451,7 +2448,7 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[c9@0 as c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 AS Int64) as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------SortExec: expr=[c9@0 DESC] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -2559,10 +2556,10 @@ GlobalLimitExec: skip=0, fetch=5 --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] ----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] ------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] ---------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)) }, 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)) }], mode=[Sorted] +--------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] ----------ProjectionExec: expr=[CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col@0 as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, inc_col@3 as inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)) }, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)) }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] 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)) }], mode=[Sorted] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)) }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)) }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 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(1)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)) }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] 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(1)), end_bound: Following(UInt64(8)) }], mode=[Sorted] +------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] 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] +--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 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(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] 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(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] ----------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] ------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true @@ -2647,8 +2644,8 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[ts@0 DESC] ----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] -------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)) }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIII @@ -2721,8 +2718,8 @@ GlobalLimitExec: skip=0, fetch=5 --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] ----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] ------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] ---------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)) }, 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)) }, 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)) }, 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)) }, 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)) }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }], mode=[Sorted] +--------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] +----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] ------------ProjectionExec: expr=[CAST(inc_col@1 AS Float64) as CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col] --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true @@ -2775,8 +2772,8 @@ GlobalLimitExec: skip=0, fetch=5 --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] ----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] ------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] ---------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)) }, 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)) }, 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)) }], mode=[Sorted] -----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)) }], mode=[Sorted] +--------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] +----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIII @@ -2820,12 +2817,11 @@ Projection: sum1, sum2, count1, count2 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] -----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] +----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] +------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] --------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] ----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] - query IIII SELECT SUM(inc_col) OVER(ORDER BY ts ASC ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING) as sum1, @@ -2867,8 +2863,8 @@ Projection: sum1, sum2, count1, count2 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] -----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }], mode=[Sorted] +----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] +------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] --------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] ----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] @@ -2967,12 +2963,12 @@ physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 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@7 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@8 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@13 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@14 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@3 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@4 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@9 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@10 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@5 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@6 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@11 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@12 as sum12] --GlobalLimitExec: skip=0, fetch=5 ----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.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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] -------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)) }, 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)) }], mode=[Linear] ---------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)) }, 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)) }], mode=[PartiallySorted([1, 0])] -----------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)) }, 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)) }], mode=[Sorted] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[PartiallySorted([0])] ---------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[PartiallySorted([0, 1])] -----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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)) }, 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: Ok(Field { name: "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", 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)) }], mode=[Sorted] +------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] +--------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])] +----------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] +------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] +--------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] +----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.a, annotated_data_infinite2.b] 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.a, annotated_data_infinite2.b] 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] ------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] --------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -3037,17 +3033,17 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST] ----ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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)) }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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=[Sorted] --------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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)) }], mode=[Sorted] +----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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=[Sorted] ------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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)) }], mode=[Sorted] +--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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] ----------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST] -------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted] +------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] --------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[Sorted] +----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] ------------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] ---------------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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)) }], mode=[Sorted] +--------------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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] ----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] ------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -3113,7 +3109,7 @@ CoalesceBatchesExec: target_batch_size=4096 --FilterExec: rn1@5 < 50 ----GlobalLimitExec: skip=0, fetch=5 ------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] # this is a negative test for asserting that window functions (other than ROW_NUMBER) @@ -3138,7 +3134,7 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] ----ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true @@ -3223,11 +3219,11 @@ Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinit --------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] ---BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] +--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] ----ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] +----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] --------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] @@ -3254,17 +3250,17 @@ Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinit --------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] ---BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] +--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] ----CoalesceBatchesExec: target_batch_size=4096 ------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST,a@1 ASC NULLS LAST --------ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------------CoalesceBatchesExec: target_batch_size=4096 --------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow }], mode=[PartiallySorted([0])] +----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] ------------------CoalesceBatchesExec: target_batch_size=4096 --------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ------------------------CoalesceBatchesExec: target_batch_size=4096 --------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST ----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] @@ -3325,10 +3321,10 @@ physical_plan GlobalLimitExec: skip=0, fetch=5 --SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] ----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] -------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 }], mode=[Sorted] +------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 }], mode=[Sorted] --------SortExec: expr=[c12@1 ASC NULLS LAST] ----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }] +------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, 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 }] --------------SortExec: expr=[c11@1 ASC NULLS LAST] ----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true @@ -3371,7 +3367,7 @@ GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] ----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] ------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] ---------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 }, 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 }], mode=[Sorted] +--------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] ----------SortExec: expr=[c12@1 ASC NULLS LAST] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true @@ -3440,9 +3436,9 @@ Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC --------TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] ---BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query TT @@ -3459,7 +3455,7 @@ Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d ------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] physical_plan ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] ---BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----CoalesceBatchesExec: target_batch_size=4096 ------FilterExec: d@1 = 0 --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true @@ -3474,7 +3470,7 @@ Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT @@ -3487,7 +3483,7 @@ Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c ----TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query I @@ -3530,7 +3526,7 @@ Limit: skip=0, fetch=5 physical_plan GlobalLimitExec: skip=0, fetch=5 --ProjectionExec: expr=[c@0 as c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] -----WindowAggExec: wdw=[NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)) }] +----WindowAggExec: wdw=[NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] ------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true query II @@ -3581,7 +3577,7 @@ Sort: multiple_ordered_table_inf.c ASC NULLS LAST physical_plan SortPreservingMergeExec: [c@3 ASC NULLS LAST] --ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] -----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow }], mode=[Linear] +----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] ------CoalesceBatchesExec: target_batch_size=4096 --------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST ----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -3915,7 +3911,7 @@ Projection: table_with_pk.sn, table_with_pk.ts, table_with_pk.currency, table_wi ----TableScan: table_with_pk projection=[sn, ts, currency, amount] physical_plan ProjectionExec: expr=[sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1] ---BoundedWindowAggExec: wdw=[SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] ----SortExec: expr=[sn@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -4035,9 +4031,9 @@ physical_plan 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@2 as sum2, sum1@1 as sum1] --GlobalLimitExec: skip=0, fetch=5 ----ProjectionExec: expr=[c3@0 as c3, sum1@3 as sum1, 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 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] -------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)) }] +------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 }] --------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] -----------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 }], mode=[Sorted] +----------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] ------------SortExec: expr=[c3@0 + c4@1 DESC] --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true @@ -4076,7 +4072,7 @@ Projection: COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETW ------TableScan: a projection=[a] physical_plan ProjectionExec: expr=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a 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 }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a 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] ----CoalesceBatchesExec: target_batch_size=4096 ------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -4099,7 +4095,7 @@ Projection: ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND ------TableScan: a projection=[a] physical_plan ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] ---BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted] +--BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ----CoalesceBatchesExec: target_batch_size=4096 ------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 From 023614e9278289086f59884ebd28a6788f856b35 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 19 Mar 2024 14:31:09 +0300 Subject: [PATCH 39/85] test changes --- .../sqllogictest/test_files/subquery.slt | 1 - .../sqllogictest/test_files/tpch/q10.slt.part | 2 + .../sqllogictest/test_files/tpch/q11.slt.part | 2 + .../sqllogictest/test_files/tpch/q19.slt.part | 1 + .../sqllogictest/test_files/tpch/q2.slt.part | 167 ++++++ .../sqllogictest/test_files/tpch/q20.slt.part | 2 + .../sqllogictest/test_files/tpch/q21.slt.part | 2 + .../sqllogictest/test_files/tpch/q22.slt.part | 1 + .../sqllogictest/test_files/tpch/q4.slt.part | 2 + .../sqllogictest/test_files/tpch/q5.slt.part | 2 + .../sqllogictest/test_files/tpch/q7.slt.part | 2 + datafusion/sqllogictest/test_files/window.slt | 495 +++++++++++++++++- 12 files changed, 677 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 66ac669c73c7..cd87c6d4babd 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -831,7 +831,6 @@ Projection: t1.t1_int ------------Aggregate: groupBy=[[t2.t2_id, Boolean(true) AS __always_true]], aggr=[[COUNT(UInt8(1)) AS COUNT(*)]] --------------TableScan: t2 projection=[t2_id] - query I rowsort select t1.t1_int from t1 where (select count(*) from t2 where t1.t1_id = t2.t2_id) < t1.t1_int ---- diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index 53fee3a52e91..b04f2538a8a0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -109,6 +109,8 @@ GlobalLimitExec: skip=0, fetch=10 ------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false + + query ITRRTTTT select c_custkey, diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index b666df91d973..837171447e7c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -128,6 +128,8 @@ GlobalLimitExec: skip=0, fetch=10 --------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false + + query IR select ps_partkey, diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 89e5e53ded72..6bd729221d04 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -84,6 +84,7 @@ ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_disco ----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false + query R select sum(l_extendedprice* (1 - l_discount)) as revenue diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 6547f93eb069..2dee58a37104 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -16,6 +16,173 @@ # specific language governing permissions and limitations # under the License. +query TT +explain select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment +from + part, + supplier, + partsupp, + nation, + region +where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 15 + and p_type like '%BRASS' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'EUROPE' +) +order by + s_acctbal desc, + n_name, + s_name, + p_partkey +limit 10; +---- +logical_plan +Limit: skip=0, fetch=10 +--Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 +----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment +------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) +--------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name +----------Inner Join: nation.n_regionkey = region.r_regionkey +------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey +--------------Inner Join: supplier.s_nationkey = nation.n_nationkey +----------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost +------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +--------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost +----------------------Inner Join: part.p_partkey = partsupp.ps_partkey +------------------------Projection: part.p_partkey, part.p_mfgr +--------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") +----------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8("%BRASS")] +------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +--------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] +----------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +------------Projection: region.r_regionkey +--------------Filter: region.r_name = Utf8("EUROPE") +----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] +--------SubqueryAlias: __scalar_sq_1 +----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey +------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] +--------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost +----------------Inner Join: nation.n_regionkey = region.r_regionkey +------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey +--------------------Inner Join: supplier.s_nationkey = nation.n_nationkey +----------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey +------------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +--------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +--------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +----------------------TableScan: nation projection=[n_nationkey, n_regionkey] +------------------Projection: region.r_regionkey +--------------------Filter: region.r_name = Utf8("EUROPE") +----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] +physical_plan +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] +--GlobalLimitExec: skip=0, fetch=10 +----SortPreservingMergeExec: [s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST], fetch=10 +------SortExec: TopK(fetch=10), expr=[s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST] +--------CoalesceBatchesExec: target_batch_size=8192 +----------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] +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------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] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------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] +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 +----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------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] +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +----------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS +------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------------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 +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +------------------------------------------------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 +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------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 +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------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 +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +----------------------------FilterExec: r_name@1 = EUROPE +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +------------CoalesceBatchesExec: target_batch_size=8192 +--------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 +----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +------------------------------------------------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 +--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +------------------------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +----------------------------CoalesceBatchesExec: target_batch_size=8192 +------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +--------------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +------------------------------------FilterExec: r_name@1 = EUROPE +--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false + + query RTTITTTT select diff --git a/datafusion/sqllogictest/test_files/tpch/q20.slt.part b/datafusion/sqllogictest/test_files/tpch/q20.slt.part index d6f2da9137c0..b68828be9ecb 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -128,6 +128,8 @@ SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 --------------------------------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 + + query TT select s_name, diff --git a/datafusion/sqllogictest/test_files/tpch/q21.slt.part b/datafusion/sqllogictest/test_files/tpch/q21.slt.part index a7a5230b310f..6b029f2ded16 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -146,6 +146,8 @@ SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] --------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 ----------------------------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 + + query TI select s_name, diff --git a/datafusion/sqllogictest/test_files/tpch/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index 3f7dcd1be990..15af6c4003a1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -103,6 +103,7 @@ SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------------------------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 select cntrycode, diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index 2776ead9dc7f..93193c43c2f8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -75,6 +75,8 @@ SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] --------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 ----------------------------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 + + query TI select o_orderpriority, diff --git a/datafusion/sqllogictest/test_files/tpch/q5.slt.part b/datafusion/sqllogictest/test_files/tpch/q5.slt.part index c6dd9f54b331..c99863a2c5b8 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -121,6 +121,8 @@ SortPreservingMergeExec: [revenue@1 DESC] ----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false + + query TR select n_name, diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 70c9fd63771e..60f0a5879937 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -139,6 +139,8 @@ SortPreservingMergeExec: [supp_nation@0 ASC NULLS LAST,cust_nation@1 ASC NULLS L ----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false + + query TTRR select supp_nation, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 9fadc7f9ca84..436524946196 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1291,6 +1291,7 @@ ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_10 --------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true + # test_window_agg_sort_reversed_plan # Only 1 SortExec was added query TT @@ -1400,6 +1401,7 @@ ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 A ----------SortExec: expr=[c9@0 DESC] ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true + query III SELECT c9, @@ -2193,7 +2195,6 @@ GlobalLimitExec: skip=0, fetch=5 --------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] ----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true - query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, SUM(c12) OVER(ORDER BY c1 GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING) as sum2 @@ -3406,6 +3407,44 @@ WITH ORDER (a ASC, b ASC) WITH ORDER (c ASC) LOCATION '../core/tests/data/window_2.csv'; +# Since column b is constant after filter b=0, +# There should be no SortExec(b ASC) in the plan +# window requirement b ASC is already satisfied by existing ordering. +query TT +EXPLAIN SELECT *, SUM(a) OVER(ORDER BY b ASC) +FROM multiple_ordered_table +where b=0 +---- +logical_plan +WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--Filter: multiple_ordered_table.b = Int32(0) +----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +physical_plan +BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +--CoalesceBatchesExec: target_batch_size=4096 +----FilterExec: b@2 = 0 +------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true + +# Since column b is constant after filter b=0, +# window requirement b ASC, d ASC can be satisfied +# by the SortExec(d ASC). We do not need to sort by [b ASC, d ASC] +query TT +EXPLAIN SELECT *, SUM(a) OVER(ORDER BY b ASC, d ASC) +FROM multiple_ordered_table +where b=0 +---- +logical_plan +WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--Filter: multiple_ordered_table.b = Int32(0) +----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +physical_plan +BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +--SortExec: expr=[d@4 ASC NULLS LAST] +----CoalesceBatchesExec: target_batch_size=4096 +------FilterExec: b@2 = 0 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true + + # Create an unbounded source where there is multiple orderings. statement ok CREATE UNBOUNDED EXTERNAL TABLE multiple_ordered_table_inf ( @@ -4307,3 +4346,457 @@ select lag(a) over (order by a ASC NULLS FIRST) as x1 NULL NULL NULL + +# Test for ignore nulls in FIRST_VALUE +statement ok +CREATE TABLE t AS VALUES (null::bigint), (3), (4); + +query I +SELECT FIRST_VALUE(column1) OVER() FROM t; +---- +NULL +NULL +NULL + +query I +SELECT FIRST_VALUE(column1) RESPECT NULLS OVER() FROM t; +---- +NULL +NULL +NULL + +query I +SELECT FIRST_VALUE(column1) IGNORE NULLS OVER() FROM t; +---- +3 +3 +3 + +statement ok +DROP TABLE t; + +# Test for ignore nulls with ORDER BY in FIRST_VALUE +statement ok +CREATE TABLE t AS VALUES (3, 4), (4, 3), (null::bigint, 1), (null::bigint, 2), (5, 5), (6, 6); + +query II +SELECT column1, column2 FROM t ORDER BY column2; +---- +NULL 1 +NULL 2 +4 3 +3 4 +5 5 +6 6 + +query II +SELECT FIRST_VALUE(column1) OVER(ORDER BY column2), column2 FROM t; +---- +NULL 1 +NULL 2 +NULL 3 +NULL 4 +NULL 5 +NULL 6 + +query II +SELECT FIRST_VALUE(column1) RESPECT NULLS OVER(ORDER BY column2), column2 FROM t; +---- +NULL 1 +NULL 2 +NULL 3 +NULL 4 +NULL 5 +NULL 6 + +query II +SELECT FIRST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2), column2 FROM t; +---- +NULL 1 +NULL 2 +4 3 +4 4 +4 5 +4 6 + +query II +SELECT FIRST_VALUE(column1)OVER(ORDER BY column2 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING), column2 FROM t; +---- +NULL 1 +NULL 2 +NULL 3 +4 4 +3 5 +5 6 + +query II +SELECT FIRST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING), column2 FROM t; +---- +NULL 1 +4 2 +4 3 +4 4 +3 5 +5 6 + +statement ok +DROP TABLE t; + +# Test for ignore nulls with ORDER BY in FIRST_VALUE with all NULL values +statement ok +CREATE TABLE t AS VALUES (null::bigint, 4), (null::bigint, 3), (null::bigint, 1), (null::bigint, 2); + +query II +SELECT FIRST_VALUE(column1) OVER(ORDER BY column2), column2 FROM t; +---- +NULL 1 +NULL 2 +NULL 3 +NULL 4 + +query II +SELECT FIRST_VALUE(column1) RESPECT NULLS OVER(ORDER BY column2), column2 FROM t; +---- +NULL 1 +NULL 2 +NULL 3 +NULL 4 + +query II +SELECT FIRST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2), column2 FROM t; +---- +NULL 1 +NULL 2 +NULL 3 +NULL 4 + +statement ok +DROP TABLE t; + +# Test for ignore nulls in LAST_VALUE +statement ok +CREATE TABLE t AS VALUES (1), (3), (null::bigint); + +query I +SELECT LAST_VALUE(column1) OVER() FROM t; +---- +NULL +NULL +NULL + +query I +SELECT LAST_VALUE(column1) RESPECT NULLS OVER() FROM t; +---- +NULL +NULL +NULL + +query I +SELECT LAST_VALUE(column1) IGNORE NULLS OVER() FROM t; +---- +3 +3 +3 + +statement ok +DROP TABLE t; + +# Test for ignore nulls with ORDER BY in LAST_VALUE +statement ok +CREATE TABLE t AS VALUES (3, 4), (4, 3), (null::bigint, 1), (null::bigint, 2), (5, 5), (6, 6); + +query II +SELECT column1, column2 FROM t ORDER BY column2 DESC NULLS LAST; +---- +6 6 +5 5 +3 4 +4 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) OVER(ORDER BY column2 DESC NULLS LAST), column2 FROM t; +---- +6 6 +5 5 +3 4 +4 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST), column2 FROM t; +---- +6 6 +5 5 +3 4 +4 3 +4 2 +4 1 + +query II +SELECT LAST_VALUE(column1) OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 6 +NULL 5 +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) RESPECT NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 6 +NULL 5 +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +4 6 +4 5 +4 4 +4 3 +4 2 +4 1 + +query II +SELECT LAST_VALUE(column1) OVER(ORDER BY column2 DESC NULLS LAST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING), column2 FROM t; +---- +5 6 +3 5 +4 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING), column2 FROM t; +---- +5 6 +3 5 +4 4 +4 3 +4 2 +NULL 1 + +statement ok +DROP TABLE t; + +# Test for ignore nulls with ORDER BY in LAST_VALUE with all NULLs +statement ok +CREATE TABLE t AS VALUES (null::bigint, 4), (null::bigint, 3), (null::bigint, 1), (null::bigint, 2); + +query II +SELECT LAST_VALUE(column1) OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) RESPECT NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT LAST_VALUE(column1) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +statement ok +DROP TABLE t; + +# Test for ignore nulls with ORDER BY in nth_VALUE +statement ok +CREATE TABLE t AS VALUES (3, 3), (4, 4), (null::bigint, 1), (null::bigint, 2), (5, 5), (6, 6); + +query I +SELECT column1 FROM t ORDER BY column2; +---- +NULL +NULL +3 +4 +5 +6 + +query I +SELECT nth_VALUE(column1, 3) OVER(ORDER BY column2) FROM t; +---- +NULL +NULL +3 +3 +3 +3 + +query I +SELECT nth_VALUE(column1, 3) IGNORE NULLS OVER(ORDER BY column2) FROM t; +---- +NULL +NULL +NULL +NULL +5 +5 + +query I +SELECT nth_VALUE(column1, 3) OVER(ORDER BY column2 ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) FROM t; +---- +3 +3 +3 +3 +3 +3 + +query I +SELECT nth_VALUE(column1, 3) IGNORE NULLS OVER(ORDER BY column2 ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) FROM t; +---- +5 +5 +5 +5 +5 +5 + +query I +SELECT nth_value(column1, 3) OVER(ORDER BY column2 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING) FROM t; +---- +NULL +3 +4 +5 +6 +NULL + +query I +SELECT nth_value(column1, 3) IGNORE NULLS OVER (ORDER BY column2 RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING) FROM t; +---- +NULL +NULL +NULL +5 +6 +NULL + +statement ok +DROP TABLE t; + +# Test for negative index in NTH_VALUE +statement ok +CREATE TABLE t AS VALUES (3, 3), (4, 4), (null::bigint, 1), (null::bigint, 2), (5, 5), (6, 6); + +query I +SELECT column1 FROM t ORDER BY column2 DESC NULLS LAST; +---- +6 +5 +4 +3 +NULL +NULL + +query I +SELECT nth_VALUE(column1, -2) OVER(ORDER BY column2 DESC NULLS LAST) FROM t; +---- +NULL +6 +5 +4 +3 +NULL + +query I +SELECT nth_VALUE(column1, -2) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST) FROM t; +---- +NULL +6 +5 +4 +4 +4 + +query I +SELECT nth_VALUE(column1, -2) OVER(ORDER BY column2 ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) FROM t; +---- +5 +5 +5 +5 +5 +5 + +query I +SELECT nth_VALUE(column1, -2) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) FROM t; +---- +4 +4 +4 +4 +4 +4 + +query I +SELECT nth_value(column1, -2) OVER (ORDER BY column2 DESC NULLS LAST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING) FROM t; +---- +6 +5 +4 +3 +NULL +NULL + +query I +SELECT nth_value(column1, -2) IGNORE NULLS OVER (ORDER BY column2 DESC NULLS LAST RANGE BETWEEN 1 PRECEDING AND 1 FOLLOWING) FROM t; +---- +6 +5 +4 +4 +NULL +NULL + +statement ok +DROP TABLE t; + +# Test for ignore nulls with ORDER BY in NTH_VALUE with all NULLs +statement ok +CREATE TABLE t AS VALUES (null::bigint, 4), (null::bigint, 3), (null::bigint, 1), (null::bigint, 2); + +query II +SELECT NTH_VALUE(column1, 2) OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT NTH_VALUE(column1, 2) RESPECT NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 4 +NULL 3 +NULL 2 +NULL 1 + +query II +SELECT NTH_VALUE(column1, 2) IGNORE NULLS OVER(ORDER BY column2 DESC NULLS LAST ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), column2 FROM t; +---- +NULL 4 +NULL 3 +NULL 2 +NULL 1 From e2a41e48cd77117b2311c9f4f64af83a4f1aa929 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 20 Mar 2024 13:39:09 +0300 Subject: [PATCH 40/85] Unify hashjoins --- .../optimize_projections.rs | 2232 +++++++---------- 1 file changed, 943 insertions(+), 1289 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 277fca94bad6..8187cfee379f 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -33,13 +33,13 @@ //! a. Merge it with its input projection if merge is beneficial. //! b. Remove the projection if it is redundant. //! c. Narrow the Projection if possible. -//! d. The projection can be nested into the source. +//! d. The projection can be nested into the input. //! e. Do nothing, otherwise. //! 2. Non-Projection node: -//! a. Schema needs pruning. Insert the necessary projections to the children. -//! b. All fields are required. Do nothing. +//! a. Schema needs pruning: Insert the necessary projection into input. +//! b. All fields are required: Do nothing. //! -//! Bottom-up Phase (now resides in map_children() implementation): +//! Bottom-up Phase (resides in with_new_children() implementation of ConcreteTreeNode): //! ---------------- //! This pass is required because modifying a plan node can change the column //! indices used by output nodes. When such a change occurs, we store the old @@ -361,7 +361,7 @@ impl ProjectionOptimizer { }; // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices_hj(projection.expr()); + let projection_index = collect_column_indices_in_proj_exprs(projection.expr()); // 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. @@ -471,70 +471,15 @@ impl ProjectionOptimizer { }) } else { let mut file_scan = csv.base_config().clone(); - let used_column_indices = projection - .expr() - .iter() - .flat_map(|(expr, _)| collect_columns(expr)) - .collect::>() - .into_iter() - .map(|col| col.index()) - .collect::>(); - - let used_csv_indices = file_scan + let required_columns = + collect_column_indices_in_proj_exprs(projection.expr()); + let file_scan_projection = file_scan .projection - .clone() - .unwrap_or((0..csv.schema().fields().len()).collect()) - .iter() - .enumerate() - .filter_map(|(idx, csv_indx)| { - if used_column_indices.contains(&idx) { - Some(*csv_indx) - } else { - None - } - }) - .collect::>(); - - let unused_indices = file_scan - .projection - .unwrap_or((0..csv.schema().fields().len()).collect()) - .iter() - .enumerate() - .filter_map(|(idx, csv_idx)| { - if used_csv_indices.contains(csv_idx) { - None - } else { - Some(idx) - } - }) - .collect::>(); - - let new_exprs = projection - .expr() - .iter() - .map(|(expr, alias)| { - let new_expr = expr - .clone() - .transform_up_mut(&mut |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::>>()?; - - file_scan.projection = Some(used_csv_indices); + .unwrap_or((0..csv.schema().fields().len()).collect()); + let (used_indices, unused_indices) = + split_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, @@ -1166,1292 +1111,538 @@ impl ProjectionOptimizer { mut self, hj: &HashJoinExec, ) -> Result { - if matches!( - hj.join_type, - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full - ) { - 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 mut hj_left_requirements = self - .required_columns - .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( - hj.on() - .iter() - .flat_map(|(left_on, _)| collect_columns(left_on)) - .collect::>(), - ); - hj_left_requirements.extend( - hj.filter() - .map(|filter| { - filter - .column_indices() - .iter() - .filter_map(|col_ind| { - if col_ind.side == JoinSide::Left { - Some(Column::new( - hj.left().schema().fields()[col_ind.index].name(), - col_ind.index, - )) - } else { - None - } - }) - .collect::>() - }) - .unwrap_or_default(), - ); - let mut hj_right_requirements = self - .required_columns - .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( - hj.on() - .iter() - .flat_map(|(_, right_on)| collect_columns(right_on)) - .collect::>(), - ); - - hj_right_requirements.extend( - hj.filter() - .map(|filter| { - filter - .column_indices() - .iter() - .filter_map(|col_ind| { - if col_ind.side == JoinSide::Right { - Some(Column::new( - hj.right().schema().fields()[col_ind.index] - .name(), - col_ind.index, - )) - } else { - None - } - }) - .collect::>() - }) - .unwrap_or_default(), - ); - - 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 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[0].clone()], - }; - - 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[1].clone()], - }; - - 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.into_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::>(); - - 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_ind| { - if col_ind.side == JoinSide::Left { - ColumnIndex { - index: left_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Left, - } - } else { - ColumnIndex { - index: right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Right, - } - } - }) - .collect(), - filter.schema().clone(), - ) - }); + match hj.join_type() { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + 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 new_projection = hj.projection.clone().map(|projection| { - projection - .iter() - .map(|ind| { - if ind < &hj.left().schema().fields().len() { - left_mapping - .iter() - .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() - } else { - right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == *ind - join_left_input_size - }) - .unwrap() - .1 - .index() - + hj_left_requirements.len() - } - }) - .collect() - }); + 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 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(), - )?; - return Ok(ProjectionOptimizer { - plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), - children_nodes: vec![new_left_node, new_right_node], - }); - } - (false, true) => { - 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[0].clone()], - }; + 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 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().cloned().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::>(); - - 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_ind| { - if col_ind.side == JoinSide::Left { - ColumnIndex { - index: left_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Left, - } - } else { - ColumnIndex { - index: right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Right, - } - } - }) - .collect(), - filter.schema().clone(), - ) + 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) }); - let new_projection = hj.projection.clone().map(|projection| { - projection - .iter() - .map(|ind| { - if ind < &hj.left().schema().fields().len() { - left_mapping - .iter() - .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() - } else { - right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == *ind - join_left_input_size - }) - .unwrap() - .1 - .index() - + hj_left_requirements.len() - } - }) - .collect() - }); + 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_hash_join = HashJoinExec::try_new( - new_left_node.plan.clone(), - self.children_nodes[1].plan.clone(), - new_on, - new_filter, - hj.join_type(), - new_projection, - *hj.partition_mode(), - hj.null_equals_null(), - )?; - let mut right_ = self.children_nodes[1].clone(); - right_.required_columns = hj_right_requirements; - return Ok(ProjectionOptimizer { - plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), - children_nodes: vec![new_left_node, right_], - }); - } - (true, false) => { - let mut ordered_hj_right_requirements = - hj_right_requirements.iter().cloned().collect_vec(); - ordered_hj_right_requirements.sort_by_key(|col| col.index()); + 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 right_projection_exprs = ordered_hj_right_requirements - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + 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: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, new_right_node], }) - .collect::>(); - - let new_right_projection = ProjectionExec::try_new( - right_projection_exprs, - hj.right().clone(), - )?; - let new_right_projection_arc = - Arc::new(new_right_projection.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[1].clone()], - }; - - 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.into_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::>(); - - 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_ind| { - if col_ind.side == JoinSide::Left { - ColumnIndex { - index: left_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Left, - } - } else { - ColumnIndex { - index: right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Right, - } - } - }) - .collect(), - filter.schema().clone(), - ) - }); - - let new_projection = hj.projection.clone().map(|projection| { - projection - .iter() - .map(|ind| { - if ind < &hj.left().schema().fields().len() { - left_mapping - .iter() - .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() - } else { - right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == *ind - join_left_input_size - }) - .unwrap() - .1 - .index() - + hj_left_requirements.len() - } - }) - .collect() - }); - - let new_hash_join = HashJoinExec::try_new( - self.children_nodes[0].plan.clone(), - new_right_node.plan.clone(), - new_on, - new_filter, - hj.join_type(), - new_projection, - *hj.partition_mode(), - hj.null_equals_null(), - )?; - let mut left_ = self.children_nodes[0].clone(); - left_.required_columns = hj_left_requirements; - return Ok(ProjectionOptimizer { - plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), - children_nodes: vec![left_, new_right_node], - }); - } - (true, true) => { - self.required_columns = HashSet::new(); - self.children_nodes.iter_mut().for_each(|c| { - c.required_columns = collect_columns_in_plan_schema(&c.plan); - }); - return Ok(self); - } - } - } - if matches!(hj.join_type, JoinType::LeftAnti | JoinType::LeftSemi) { - let join_left_input_size = hj.left().schema().fields().len(); - let join_projection = hj - .projection - .clone() - .unwrap_or((0..hj.left().schema().fields().len()).collect()); - let mut hj_left_requirements = self - .required_columns - .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( - hj.on() - .iter() - .flat_map(|(left_on, _)| collect_columns(left_on)) - .collect::>(), - ); - hj_left_requirements.extend( - hj.filter() - .map(|filter| { - filter - .column_indices() - .iter() - .filter_map(|col_ind| { - if col_ind.side == JoinSide::Left { - Some(Column::new( - hj.left().schema().fields()[col_ind.index].name(), - col_ind.index, - )) - } else { - None - } - }) - .collect::>() - }) - .unwrap_or_default(), - ); - let mut hj_right_requirements = HashSet::new(); - hj_right_requirements.extend( - hj.on() - .iter() - .flat_map(|(_, right_on)| collect_columns(right_on)) - .collect::>(), - ); - hj_right_requirements.extend( - hj.filter() - .map(|filter| { - filter - .column_indices() - .iter() - .filter_map(|col_ind| { - if col_ind.side == JoinSide::Right { - Some(Column::new( - hj.right().schema().fields()[col_ind.index] - .name(), - col_ind.index, - )) - } else { - None - } - }) - .collect::>() - }) - .unwrap_or_default(), - ); - - 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) - }); + (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 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) - }); + 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, + ); - match (keep_left_same, keep_right_same) { - (false, false) => { - 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[0].clone()], - }; + 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(), + )?; - 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, right_node], }) - .collect::>(); - let new_right_projection = ProjectionExec::try_new( - right_projection_exprs, - hj.right().clone(), - )?; - let new_right_projection_arc = - Arc::new(new_right_projection.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[1].clone()], - }; + } + (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 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.into_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::>(); - - 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_ind| { - if col_ind.side == JoinSide::Left { - ColumnIndex { - index: left_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Left, - } - } else { - ColumnIndex { - index: right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Right, - } - } - }) - .collect(), - filter.schema().clone(), - ) - }); + 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_projection = hj.projection.clone().map(|projection| { - projection - .iter() - .map(|ind| { - if ind < &hj.left().schema().fields().len() { - left_mapping - .iter() - .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() - } else { - right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == *ind - join_left_input_size - }) - .unwrap() - .1 - .index() - + hj_left_requirements.len() - } - }) - .collect() - }); + 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(), + )?; - 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(), - )?; - return Ok(ProjectionOptimizer { - plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), - children_nodes: vec![new_left_node, new_right_node], - }); - } - (false, true) => { - 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), + required_columns: HashSet::new(), + schema_mapping: HashMap::new(), + children_nodes: vec![left_node, new_right_node], }) - .collect::>(); - let new_left_projection = ProjectionExec::try_new( - left_projection_exprs, - hj.left().clone(), - )?; - let new_left_projection_arc = - Arc::new(new_left_projection.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[0].clone()], - }; - - 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.into_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::>(); - - 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_ind| { - if col_ind.side == JoinSide::Left { - ColumnIndex { - index: left_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Left, - } - } else { - ColumnIndex { - index: right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Right, - } - } - }) - .collect(), - filter.schema().clone(), - ) - }); - - let new_projection = hj.projection.clone().map(|projection| { - projection - .iter() - .map(|ind| { - if ind < &hj.left().schema().fields().len() { - left_mapping - .iter() - .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() - } else { - right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == *ind - join_left_input_size - }) - .unwrap() - .1 - .index() - + hj_left_requirements.len() - } - }) - .collect() - }); - - let new_hash_join = HashJoinExec::try_new( - new_left_node.plan.clone(), - hj.right().clone(), - new_on, - new_filter, - hj.join_type(), - new_projection, - *hj.partition_mode(), - hj.null_equals_null(), - )?; - return Ok(ProjectionOptimizer { - plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), - children_nodes: vec![ - new_left_node, - self.children_nodes[1].clone(), - ], - }); + } + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }); + Ok(self) + } } - (true, false) => { - 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 - .iter() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), - children_nodes: vec![self.children_nodes[1].clone()], - }; + } + JoinType::LeftSemi | JoinType::LeftAnti => { + let join_left_input_size = hj.left().schema().fields().len(); + let join_projection = hj + .projection + .clone() + .unwrap_or((0..hj.left().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 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.into_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::>(); - - 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_ind| { - if col_ind.side == JoinSide::Left { - ColumnIndex { - index: left_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Left, - } - } else { - ColumnIndex { - index: right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == col_ind.index - }) - .unwrap() - .1 - .index(), - side: JoinSide::Right, - } - } - }) - .collect(), - filter.schema().clone(), - ) - }); + 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 new_projection = hj.projection.clone().map(|projection| { - projection - .iter() - .map(|ind| { - if ind < &hj.left().schema().fields().len() { - left_mapping - .iter() - .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() - } else { - right_mapping - .iter() - .find(|(initial, _)| { - initial.index() == *ind - join_left_input_size - }) - .unwrap() - .1 - .index() - + hj_left_requirements.len() - } - }) - .collect() - }); + 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 new_hash_join = HashJoinExec::try_new( - hj.left().clone(), - new_right_node.plan.clone(), - new_on, - new_filter, - hj.join_type(), - new_projection, - *hj.partition_mode(), - hj.null_equals_null(), - )?; - return Ok(ProjectionOptimizer { - plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), - children_nodes: vec![ - self.children_nodes[0].clone(), - new_right_node, - ], - }); - } - (true, true) => { - self.required_columns = HashSet::new(); - self.children_nodes.iter_mut().for_each(|c| { - c.required_columns = collect_columns_in_plan_schema(&c.plan); + 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) }); - return Ok(self); - } - } - } - let left_size = hj.left().schema().fields().len(); - // HashJoinExec extends the requirements with the columns in its equivalence and non-equivalence conditions. - match hj.join_type() { - JoinType::RightAnti | JoinType::RightSemi => { - self.required_columns = - update_right_requirements(self.required_columns, left_size); - } - _ => {} - } - self.required_columns - .extend(collect_columns_in_join_conditions( - hj.on(), - hj.filter(), - left_size, - self.children_nodes[0].plan.schema(), - self.children_nodes[1].plan.schema(), - )); - let (analyzed_join_left, analyzed_join_right) = analyze_requirements_of_joins( - hj.left(), - hj.right(), - &self.required_columns, - left_size, - ); - - match hj.join_type() { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - match ( - all_columns_required(&analyzed_join_left), - all_columns_required(&analyzed_join_right), - ) { - // We need two projections on top of both children. + match (keep_left_same, keep_right_same) { (false, false) => { - let new_on = update_equivalence_conditions( - hj.on(), - &analyzed_join_left, - &analyzed_join_right, + 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_filter = update_non_equivalence_conditions( - hj.filter(), - &analyzed_join_left, - &analyzed_join_right, + + 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_left_child, new_right_child, schema_mapping) = self - .insert_multi_projections_below_join( - left_size, - analyzed_join_left, - analyzed_join_right, - )?; - let plan = Arc::new(HashJoinExec::try_new( - new_left_child.plan.clone(), - new_right_child.plan.clone(), + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + new_right_node.plan.clone(), new_on, new_filter, hj.join_type(), - None, + new_projection, *hj.partition_mode(), hj.null_equals_null(), - )?) as _; - - self = ProjectionOptimizer { - plan, + )?; + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), required_columns: HashSet::new(), - schema_mapping, - children_nodes: vec![new_left_child, new_right_child], - } + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }) } (false, true) => { - let required_columns = self.required_columns.clone(); - let mut right_child = self.children_nodes.swap_remove(1); - - let new_on = update_equivalence_conditions( - hj.on(), - &analyzed_join_left, - &HashMap::new(), - ); - let new_filter = update_non_equivalence_conditions( - hj.filter(), - &analyzed_join_right, - &HashMap::new(), + 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_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 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 plan = Arc::new(HashJoinExec::try_new( - new_left_child.plan.clone(), - right_child.plan.clone(), + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + right_node.plan.clone(), new_on, new_filter, hj.join_type(), - None, + new_projection, *hj.partition_mode(), hj.null_equals_null(), - )?) 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, + )?; + + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), required_columns: HashSet::new(), - schema_mapping: left_schema_mapping, - children_nodes: vec![new_left_child, right_child], - } + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, right_node], + }) } (true, false) => { - let required_columns = self.required_columns.clone(); - let mut left_child = self.children_nodes.swap_remove(0); - let new_on = update_equivalence_conditions( - hj.on(), - &HashMap::new(), - &analyzed_join_right, + 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_filter = update_non_equivalence_conditions( - hj.filter(), - &HashMap::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 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 plan = Arc::new(HashJoinExec::try_new( - left_child.plan.clone(), - new_right_child.plan.clone(), + let new_hash_join = HashJoinExec::try_new( + left_node.plan.clone(), + new_right_node.plan.clone(), new_on, new_filter, hj.join_type(), - None, + new_projection, *hj.partition_mode(), hj.null_equals_null(), - )?) as _; + )?; - self = ProjectionOptimizer { - plan, + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), required_columns: HashSet::new(), - schema_mapping: right_schema_mapping, - children_nodes: vec![left_child, new_right_child], - } + schema_mapping: HashMap::new(), + children_nodes: vec![left_node, new_right_node], + }) } - // All columns are required. (true, true) => { self.required_columns = HashSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); - }) + }); + Ok(self) } } } - JoinType::LeftAnti | JoinType::LeftSemi => { - match all_columns_required(&analyzed_join_left) { - false => { - let mut right_child = self.children_nodes.swap_remove(1); - let new_on = update_equivalence_conditions( - hj.on(), - &analyzed_join_left, - &HashMap::new(), - ); - let new_filter = update_non_equivalence_conditions( - hj.filter(), - &analyzed_join_left, - &HashMap::new(), + JoinType::RightSemi | JoinType::RightAnti => { + let join_left_input_size = hj.left().schema().fields().len(); + 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, + 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_left_child, left_schema_mapping) = - self.insert_projection_below_left_child(analyzed_join_left)?; + 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 plan = Arc::new(HashJoinExec::try_new( - new_left_child.plan.clone(), - right_child.plan.clone(), + let new_hash_join = HashJoinExec::try_new( + new_left_node.plan.clone(), + new_right_node.plan.clone(), new_on, new_filter, hj.join_type(), - None, + new_projection, *hj.partition_mode(), hj.null_equals_null(), - )?) as _; - - right_child.required_columns = analyzed_join_right - .into_iter() - .filter_map( - |(column, used)| if used { Some(column) } else { None }, - ) - .collect(); - self = ProjectionOptimizer { - plan, + )?; + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), required_columns: HashSet::new(), - schema_mapping: left_schema_mapping, - children_nodes: vec![new_left_child, right_child], - } - } - true => { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan); - self.children_nodes[1].required_columns = analyzed_join_right - .into_iter() - .filter_map( - |(column, used)| if used { Some(column) } else { None }, - ) - .collect() + schema_mapping: HashMap::new(), + children_nodes: vec![new_left_node, new_right_node], + }) } - } - } - JoinType::RightAnti | JoinType::RightSemi => { - match all_columns_required(&analyzed_join_right) { - false => { - let mut left_child = self.children_nodes.swap_remove(0); - let new_on = update_equivalence_conditions( - hj.on(), - &HashMap::new(), - &analyzed_join_right, + (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: HashSet::new(), + schema_mapping: HashMap::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_filter = update_non_equivalence_conditions( - hj.filter(), - &HashMap::new(), - &analyzed_join_right, + + 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_right_child, right_schema_mapping) = self - .insert_projection_below_right_child(analyzed_join_right)?; - let plan = Arc::new(HashJoinExec::try_new( - left_child.plan.clone(), - new_right_child.plan.clone(), + let new_hash_join = HashJoinExec::try_new( + left_node.plan.clone(), + new_right_node.plan.clone(), new_on, new_filter, hj.join_type(), - None, + new_projection, *hj.partition_mode(), hj.null_equals_null(), - )?) as _; + )?; - left_child.required_columns = analyzed_join_left - .into_iter() - .filter_map( - |(column, used)| if used { Some(column) } else { None }, - ) - .collect(); - self = ProjectionOptimizer { - plan, + Ok(ProjectionOptimizer { + plan: Arc::new(new_hash_join), required_columns: HashSet::new(), - schema_mapping: right_schema_mapping, - children_nodes: vec![left_child, new_right_child], - } + schema_mapping: HashMap::new(), + children_nodes: vec![left_node, new_right_node], + }) } - 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); + (true, true) => { + self.required_columns = HashSet::new(); + self.children_nodes.iter_mut().for_each(|c| { + c.required_columns = collect_columns_in_plan_schema(&c.plan); + }); + Ok(self) } } } } - Ok(self) } fn try_insert_below_nested_loop_join( @@ -4257,7 +3448,7 @@ impl ProjectionOptimizer { self.plan = self.plan.with_new_children(vec![new_child.plan.clone()])?; self.children_nodes = vec![new_child]; } else { - unreachable!() + return Ok(Transformed::no(self)); } } else { let res = self.plan.clone().with_new_children( @@ -4964,8 +4155,10 @@ fn collect_columns_in_join_conditions( .collect::>() } -fn collect_column_indices_hj(exprs: &[(Arc, String)]) -> Vec { - // Collect indices and remove duplicates. +/// 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)) @@ -4973,6 +4166,206 @@ fn collect_column_indices_hj(exprs: &[(Arc, String)]) -> Vec>() } +fn split_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) +} + +fn collect_hj_left_requirements( + all_requirements: &HashSet, + join_projection: &[usize], + join_left_input_size: usize, + join_left_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> HashSet { + 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>, +) -> HashSet { + let mut hj_left_requirements = HashSet::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: &HashSet, + join_projection: &[usize], + join_left_input_size: usize, + join_right_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> HashSet { + 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: &HashSet, + join_projection: &[usize], + join_left_input_size: usize, + join_right_schema: SchemaRef, + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, +) -> HashSet { + 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 +} + #[derive(Debug, PartialEq)] enum RewriteState { /// The expression is unchanged. @@ -5309,6 +4702,277 @@ fn update_right_requirements( .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_mut(&mut |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: &HashSet, + hj_right_requirements: &HashSet, + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![children.swap_remove(0)], + }; + + Ok((new_left_node, new_right_node)) +} + +fn update_hj_left_child( + hj_left_requirements: &HashSet, + hj_right_requirements: &HashSet, + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![children.swap_remove(0)], + }; + + let mut right_node = children.swap_remove(0); + right_node.required_columns = hj_right_requirements.clone(); + + Ok((new_left_node, right_node)) +} + +fn update_hj_right_child( + hj_left_requirements: &HashSet, + hj_right_requirements: &HashSet, + 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 + .iter() + .map(|req| { + let name = req.name().to_owned(); + (Arc::new(req.clone()) 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.clone()) 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: HashMap::new(), + children_nodes: vec![children.swap_remove(1)], + }; + + let mut left_node = children.swap_remove(0); + left_node.required_columns = hj_left_requirements.clone(); + + Ok((left_node, new_right_node)) +} + +fn update_hj_children_mapping( + hj_left_requirements: &HashSet, + hj_right_requirements: &HashSet, +) -> (HashMap, HashMap) { + 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: HashSet, + left_mapping: HashMap, + right_mapping: HashMap, + join_left_input_size: usize, +) -> Option> { + projection.clone().map(|projection| { + projection + .iter() + .map(|ind| { + if ind < &hj_left_schema.fields().len() { + left_mapping + .iter() + .find(|(initial, _)| initial.index() == *ind) + .unwrap() + .1 + .index() + } else { + right_mapping + .iter() + .find(|(initial, _)| { + initial.index() == *ind - join_left_input_size + }) + .unwrap() + .1 + .index() + + hj_left_requirements.len() + } + }) + .collect() + }) +} + +/// Rewrites a filter execution plan with updated column indices. +/// +/// This function updates the column indices in a filter's predicate based on a provided mapping. +/// It creates a new `FilterExec` with the updated predicate. +/// +/// # Arguments +/// * `predicate` - The predicate expression of the filter. +/// * `input_plan` - The input execution plan on which the filter is applied. +/// * `mapping` - A hashmap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `FilterExec` wrapped in an `Arc`. +fn rewrite_filter( + predicate: &Arc, + input_plan: Arc, + mapping: &HashMap, +) -> Result> { + FilterExec::try_new(update_column_index(predicate, mapping), input_plan) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_hj_filter( + filter: Option<&JoinFilter>, + left_mapping: &HashMap, + right_mapping: &HashMap, +) -> 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) + .unwrap() + .1 + .index(), + side: JoinSide::Left, + } + } else { + ColumnIndex { + index: right_mapping + .iter() + .find(|(initial, _)| initial.index() == col_ind.index) + .unwrap() + .1 + .index(), + side: JoinSide::Right, + } + } + }) + .collect(), + filter.schema().clone(), + ) + }) +} + /// Rewrites a projection execution plan with updated column indices. /// /// This function updates the column indices in a projection based on a provided mapping. @@ -5338,27 +5002,6 @@ fn rewrite_projection( .map(|plan| Arc::new(plan) as _) } -/// Rewrites a filter execution plan with updated column indices. -/// -/// This function updates the column indices in a filter's predicate based on a provided mapping. -/// It creates a new `FilterExec` with the updated predicate. -/// -/// # Arguments -/// * `predicate` - The predicate expression of the filter. -/// * `input_plan` - The input execution plan on which the filter is applied. -/// * `mapping` - A hashmap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `FilterExec` wrapped in an `Arc`. -fn rewrite_filter( - predicate: &Arc, - input_plan: Arc, - mapping: &HashMap, -) -> Result> { - FilterExec::try_new(update_column_index(predicate, mapping), input_plan) - .map(|plan| Arc::new(plan) as _) -} - /// Rewrites a repartition execution plan with updated column indices. /// /// Updates the partitioning expressions in a repartition plan based on the provided column index mappings. @@ -5432,6 +5075,17 @@ fn rewrite_sort_preserving_merge( ) as _) } +fn rewrite_hash_join( + nlj: &NestedLoopJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + left_mapping: &HashMap, + right_mapping: &HashMap, + left_size: usize, +) -> Result> { + todo!() +} + fn rewrite_nested_loop_join( nlj: &NestedLoopJoinExec, left_input_plan: Arc, From b20939155dab88c2da092c543aacfc8876e617c8 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 20 Mar 2024 15:46:00 +0300 Subject: [PATCH 41/85] Addressing mete's latest feedback --- .../optimize_projections.rs | 162 ++++++++---------- 1 file changed, 73 insertions(+), 89 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 8187cfee379f..315f2537adae 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -253,11 +253,9 @@ impl ProjectionOptimizer { projected_exprs.push((expr, alias.clone())); } - let new_plan = ProjectionExec::try_new( - projected_exprs.clone(), - child_projection.input().clone(), - ) - .map(|e| Arc::new(e) as _)?; + let new_plan = + ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) + .map(|e| Arc::new(e) as _)?; Ok(Transformed::yes(Self { plan: new_plan, @@ -1023,7 +1021,7 @@ impl ProjectionOptimizer { } // Left child needs a projection. (false, true) => { - let required_columns = self.required_columns.clone(); + let required_columns = mem::take(&mut self.required_columns); let mut right_child = self.children_nodes.swap_remove(1); let (new_left_child, mut left_schema_mapping) = @@ -1052,7 +1050,7 @@ impl ProjectionOptimizer { } // Right child needs a projection. (true, false) => { - let required_columns = self.required_columns.clone(); + let required_columns = mem::take(&mut self.required_columns); let mut left_child = self.children_nodes.swap_remove(0); let (new_right_child, mut right_schema_mapping) = self.insert_projection_below_right_child(analyzed_join_right)?; @@ -1483,7 +1481,6 @@ impl ProjectionOptimizer { let hj_right_requirements = collect_right_hj_right_requirements( &self.required_columns, &join_projection, - join_left_input_size, hj.right().schema(), hj.on(), hj.filter(), @@ -1707,7 +1704,7 @@ impl ProjectionOptimizer { } } (false, true) => { - let required_columns = self.required_columns.clone(); + 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(), @@ -1742,7 +1739,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let required_columns = self.required_columns.clone(); + 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(), @@ -1943,7 +1940,7 @@ impl ProjectionOptimizer { } } (false, true) => { - let required_columns = self.required_columns.clone(); + 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(), @@ -1986,7 +1983,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let required_columns = self.required_columns.clone(); + 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(), @@ -2221,7 +2218,7 @@ impl ProjectionOptimizer { } } (false, true) => { - let required_columns = self.required_columns.clone(); + 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(), @@ -2265,7 +2262,7 @@ impl ProjectionOptimizer { } } (true, false) => { - let required_columns = self.required_columns.clone(); + 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(), @@ -2643,15 +2640,14 @@ impl ProjectionOptimizer { .collect(); } } else { - self.children_nodes[0].required_columns = self - .required_columns - .iter() - .filter(|col| { - col.index() - < w_agg.schema().fields().len() - w_agg.window_expr().len() - }) - .cloned() - .collect(); + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns) + .into_iter() + .filter(|col| { + col.index() + < w_agg.schema().fields().len() - w_agg.window_expr().len() + }) + .collect(); } Ok(self) } @@ -2683,16 +2679,15 @@ impl ProjectionOptimizer { .iter() .any(|expr| expr.clone().with_new_expressions(vec![]).is_none()) { - self.children_nodes[0].required_columns = self - .required_columns - .iter() - .filter(|col| { - col.index() - < bw_agg.schema().fields().len() - - bw_agg.window_expr().len() - }) - .cloned() - .collect(); + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns) + .into_iter() + .filter(|col| { + col.index() + < bw_agg.schema().fields().len() + - bw_agg.window_expr().len() + }) + .collect(); return Ok(self); } let (new_child, schema_mapping, window_usage) = @@ -2751,15 +2746,14 @@ impl ProjectionOptimizer { .collect(); } } else { - self.children_nodes[0].required_columns = self - .required_columns - .iter() - .filter(|col| { - col.index() - < bw_agg.schema().fields().len() - bw_agg.window_expr().len() - }) - .cloned() - .collect(); + self.children_nodes[0].required_columns = + mem::take(&mut self.required_columns) + .into_iter() + .filter(|col| { + col.index() + < bw_agg.schema().fields().len() - bw_agg.window_expr().len() + }) + .collect(); } Ok(self) } @@ -3066,8 +3060,8 @@ impl ProjectionOptimizer { fn index_updater(mut self: ProjectionOptimizer) -> Result> { let mut all_mappings = self .children_nodes - .iter() - .map(|node| node.schema_mapping.clone()) + .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. @@ -3648,7 +3642,7 @@ fn satisfy_initial_schema( Ok(po) } else { let mut initial_requirements_ordered = - initial_requirements.clone().into_iter().collect_vec(); + initial_requirements.into_iter().collect_vec(); initial_requirements_ordered.sort_by_key(|expr| expr.index()); let projected_exprs = initial_requirements_ordered .into_iter() @@ -4328,7 +4322,6 @@ fn collect_hj_right_requirements( fn collect_right_hj_right_requirements( all_requirements: &HashSet, join_projection: &[usize], - join_left_input_size: usize, join_right_schema: SchemaRef, on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, @@ -4742,16 +4735,15 @@ fn update_hj_children( 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 - .iter() + .into_iter() .map(|req| { let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + (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.clone()) as Arc; + 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, @@ -4764,17 +4756,17 @@ fn update_hj_children( 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 - .iter() + .into_iter() .map(|req| { let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + (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.clone()) as 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 { @@ -4797,16 +4789,15 @@ fn update_hj_left_child( 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 - .iter() + .into_iter() .map(|req| { let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + (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.clone()) as Arc; + 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, @@ -4831,16 +4822,16 @@ fn update_hj_right_child( 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 - .iter() + .into_iter() .map(|req| { let name = req.name().to_owned(); - (Arc::new(req.clone()) as Arc, name) + (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.clone()) as 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 { @@ -4885,7 +4876,7 @@ fn update_hj_projection( right_mapping: HashMap, join_left_input_size: usize, ) -> Option> { - projection.clone().map(|projection| { + projection.map(|projection| { projection .iter() .map(|ind| { @@ -4893,18 +4884,16 @@ fn update_hj_projection( left_mapping .iter() .find(|(initial, _)| initial.index() == *ind) - .unwrap() - .1 - .index() + .map(|(_, target)| target.index()) + .unwrap_or(*ind) } else { right_mapping .iter() .find(|(initial, _)| { initial.index() == *ind - join_left_input_size }) - .unwrap() - .1 - .index() + .map(|(_, target)| target.index()) + .unwrap_or(*ind) + hj_left_requirements.len() } }) @@ -4950,9 +4939,8 @@ fn rewrite_hj_filter( index: left_mapping .iter() .find(|(initial, _)| initial.index() == col_ind.index) - .unwrap() - .1 - .index(), + .map(|(_, target)| target.index()) + .unwrap_or(col_ind.index), side: JoinSide::Left, } } else { @@ -4960,9 +4948,8 @@ fn rewrite_hj_filter( index: right_mapping .iter() .find(|(initial, _)| initial.index() == col_ind.index) - .unwrap() - .1 - .index(), + .map(|(_, target)| target.index()) + .unwrap_or(col_ind.index), side: JoinSide::Right, } } @@ -5075,17 +5062,6 @@ fn rewrite_sort_preserving_merge( ) as _) } -fn rewrite_hash_join( - nlj: &NestedLoopJoinExec, - left_input_plan: Arc, - right_input_plan: Arc, - left_mapping: &HashMap, - right_mapping: &HashMap, - left_size: usize, -) -> Result> { - todo!() -} - fn rewrite_nested_loop_join( nlj: &NestedLoopJoinExec, left_input_plan: Arc, @@ -5427,7 +5403,7 @@ mod tests { use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_expr::{ColumnarValue, Operator}; + use datafusion_expr::{Operator, ScalarFunctionDefinition}; use datafusion_physical_expr::expressions::{ BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, }; @@ -5511,7 +5487,9 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + ScalarFunctionDefinition::Name( + "dummy".to_owned().into_boxed_str().into(), + ), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5578,7 +5556,9 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + ScalarFunctionDefinition::Name( + "dummy".to_owned().into_boxed_str().into(), + ), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5648,7 +5628,9 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + ScalarFunctionDefinition::Name( + "dummy".to_owned().into_boxed_str().into(), + ), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5715,7 +5697,9 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - Arc::new(|_: &[ColumnarValue]| unimplemented!("not implemented")), + ScalarFunctionDefinition::Name( + "dummy".to_owned().into_boxed_str().into(), + ), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b_new", 1)), From a1f7d307fba274fbf78d699b91e94067b82818f4 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 21 Mar 2024 10:53:36 +0300 Subject: [PATCH 42/85] Fixing the bug during benchmarks --- .../optimize_projections.rs | 167 +++++++++++++++--- 1 file changed, 139 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 315f2537adae..913b0e3db513 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3170,7 +3170,10 @@ impl ProjectionOptimizer { } else if let Some(hj) = plan_any.downcast_ref::() { let left_mapping = all_mappings.swap_remove(0); let right_mapping = all_mappings.swap_remove(0); - let projection = hj.projection.clone(); + 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( @@ -3208,34 +3211,142 @@ impl ProjectionOptimizer { filter.schema().clone(), ) }); - let index_mapping = left_mapping - .into_iter() - .map(|(col1, col2)| (col1.index(), col2.index())) - .chain(right_mapping.into_iter().map(|(col1, col2)| { - ( - col1.index() + hj.children()[0].schema().fields().len(), - col2.index() - + self.children_nodes[0].plan.schema().fields().len(), + + 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() + + hj.children()[0].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(), ) - })) - .collect::>(); - let new_projection = projection.map(|mut prj| { - prj.iter_mut() - .for_each(|idx| *idx = *index_mapping.get(idx).unwrap_or(idx)); - prj - }); - self.plan = HashJoinExec::try_new( - self.children_nodes[0].plan.clone(), - self.children_nodes[1].plan.clone(), - new_on, - new_filter, - hj.join_type(), - new_projection, - *hj.partition_mode(), - hj.null_equals_null(), - ) - .map(|plan| Arc::new(plan) as _)?; - self.schema_mapping = HashMap::new(); + .map(|plan| Arc::new(plan) as _)?; + self.schema_mapping = left_mapping + .into_iter() + .chain(right_mapping.iter().map(|(col1, col2)| { + ( + Column::new( + col1.name(), + col1.index() + + hj.children()[0].schema().fields().len(), + ), + Column::new( + col2.name(), + col2.index() + + self.children_nodes[0] + .plan + .schema() + .fields() + .len(), + ), + ) + })) + .collect::>(); + } + 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.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.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; + } + } } else if let Some(nlj) = plan_any.downcast_ref::() { let left_size = self.children_nodes[0].plan.schema().fields().len(); let left_mapping = all_mappings.swap_remove(0); From f5e58e14bafa29eedb7c9cafbff976cdbac56d3c Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 21 Mar 2024 13:04:35 +0300 Subject: [PATCH 43/85] Update cte.slt --- datafusion/sqllogictest/test_files/cte.slt | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 50c88e41959f..5fb21e124773 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -709,8 +709,8 @@ RecursiveQueryExec: name=recursive_cte, is_distinct=false ----CrossJoinExec ------CoalescePartitionsExec --------CoalesceBatchesExec: target_batch_size=8182 -----------FilterExec: val@0 < 2 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------WorkTableExec: name=recursive_cte -------ProjectionExec: expr=[2 as val] ---------PlaceholderRowExec +----------ProjectionExec: expr=[] +------------FilterExec: val@0 < 2 +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------WorkTableExec: name=recursive_cte +------PlaceholderRowExec From 7f77d502955809e91261bb9b25a1d8944478b4db Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 22 Mar 2024 17:04:58 +0300 Subject: [PATCH 44/85] tests added --- .../optimize_projections.rs | 357 +++++++++++++++++- 1 file changed, 342 insertions(+), 15 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 913b0e3db513..f0e6f39e2724 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -79,6 +79,7 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::displayable; use datafusion_physical_plan::insert::FileSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, @@ -96,6 +97,13 @@ use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use itertools::Itertools; +fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) +} + /// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary /// fields for column requirements and changed indices of columns. #[derive(Debug, Clone)] @@ -3949,20 +3957,12 @@ fn is_projection_removable(projection: &ProjectionExec) -> bool { /// 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().any(|expr| { - expr.clone() - .with_new_expressions(expr.expressions()) - .is_some() - }) + aggr_expr.iter().all(|expr| expr.expressions().is_empty()) } /// Tries to rewrite the [`WindowExpr`] with the existing expressions to keep on optimization. fn is_window_expr_rewritable(window_expr: &[Arc<(dyn WindowExpr)>]) -> bool { - window_expr.iter().any(|expr| { - expr.clone() - .with_new_expressions(expr.expressions()) - .is_some() - }) + window_expr.iter().all(|expr| expr.expressions().is_empty()) } /// Updates a source provider's projected columns according to the given @@ -4192,7 +4192,7 @@ fn collect_alias_free_columns( /// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. /// /// # Arguments -/// * `plan`: Reference to an Arc of a dynamic ExecutionPlan trait object. +/// * `plan`: Reference to an Arc of an ExecutionPlan trait object. /// /// # Returns /// A `HashSet` containing all columns from the plan's schema. @@ -5514,9 +5514,13 @@ mod tests { use datafusion_common::config::ConfigOptions; use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_expr::{Operator, ScalarFunctionDefinition}; + use datafusion_expr::{Operator, ScalarFunctionDefinition, WindowFrame}; use datafusion_physical_expr::expressions::{ - BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, + rank, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, RowNumber, + Sum, + }; + use datafusion_physical_expr::window::{ + BuiltInWindowExpr, BuiltInWindowFunctionExpr, }; use datafusion_physical_expr::{ Partitioning, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, @@ -5525,6 +5529,7 @@ mod tests { HashJoinExec, PartitionMode, SymmetricHashJoinExec, }; use datafusion_physical_plan::union::UnionExec; + use datafusion_physical_plan::InputOrderMode; fn create_simple_csv_exec() -> Arc { let schema = Arc::new(Schema::new(vec![ @@ -6534,8 +6539,8 @@ mod tests { FilterExec(sum > 0): |sum@0 | ProjectionExec: |c@0+x@1 as sum | SortExec(c@0, x@1): |c@0 |x@1 | - ProjectionExec: |c@2 |a@0 as x | - CsvExec: |a |b |c |d |e | + ProjectionExec: |c@1 |a@0 as x | + CsvExec: |a |c | */ let csv = create_simple_csv_exec(); @@ -6628,4 +6633,326 @@ mod tests { assert_eq!(get_plan_string(&after_optimize), expected); Ok(()) } + + #[test] + fn test_optimize_projections_left_anti() -> Result<()> { + /* + INITIAL PLAN: + AggregateExec(gb(a@2), sum(rownumber@1)): |gb(a) |sum(rownumber) | + HashJoin(a@1=b@1, rownumber@4; + + 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]", + " 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]", + " 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<()> { + /* + INITIAL PLAN: + AggregateExec(gb(a@2), sum(rank@1)): |gb(a) |sum(rank) | + HashJoin(a@1=b@1, rownumber@4; + + let initial = get_plan_string(&aggregate); + + let expected_initial = [ + "AggregateExec: mode=Single, gby=[a@2 as a], aggr=[SUM(RANK())]", + " 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]", + " 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(RANK())]", + " HashJoinExec: mode=Auto, join_type=RightSemi, on=[(a@0, b@1)], filter=ROW_NUMBER()@0 < RANK@1, projection=[a@0, a@0, a@0]", + " ProjectionExec: expr=[a@0 as a, ROW_NUMBER()@2 as ROW_NUMBER()]", + " SortExec: expr=[ROW_NUMBER()@2 ASC,d@1 ASC]", + " 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(()) + } } From 9d11497ab063b12e8ac8bf259244bb97c8876747 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 22 Mar 2024 17:55:13 +0300 Subject: [PATCH 45/85] checked till try_remove --- .../optimize_projections.rs | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index f0e6f39e2724..a10016e0fe69 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -280,14 +280,15 @@ impl ProjectionOptimizer { /// 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 + // 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) = collect_alias_free_columns(projection_exec.expr()) + let Some(projection_columns) = + try_collect_alias_free_columns(projection_exec.expr()) else { return Ok(Transformed::no(self)); }; @@ -451,7 +452,9 @@ impl ProjectionOptimizer { }; // 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) = collect_alias_free_columns(projection.expr()) { + if let Some(projection_columns) = + try_collect_alias_free_columns(projection.expr()) + { let mut file_scan = csv.base_config().clone(); let new_projections = new_projections_for_columns( &projection_columns, @@ -3591,7 +3594,7 @@ impl ProjectionOptimizer { // Is the projection really required? First, we need to // have all column expression in the projection for removal. - let Some(projection_columns) = collect_alias_free_columns(projection.expr()) + let Some(projection_columns) = try_collect_alias_free_columns(projection.expr()) else { return Ok(self); }; @@ -3754,9 +3757,7 @@ fn satisfy_initial_schema( mut po: ProjectionOptimizer, initial_requirements: HashSet, ) -> Result { - if collect_columns_in_plan_schema(&po.plan) == initial_requirements - && po.schema_mapping.is_empty() - { + if po.schema_mapping.is_empty() { // The initial schema is already satisfied, no further action required. Ok(po) } else { @@ -4087,6 +4088,7 @@ fn removed_column_count( /// 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" @@ -4095,7 +4097,7 @@ fn removed_column_count( /// /// # Arguments /// * `requirements`: Reference to a `HashSet` representing the parent's column requirements. -/// * `projection_columns`: Slice of `Column` representing the columns in the projection. +/// * `projection_columns`: Slice of `Column` representing the column expressions in the projection. /// /// # Returns /// A `HashSet` with updated column indices reflecting the child's perspective. @@ -4176,7 +4178,7 @@ fn collect_left_used_columns( /// 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 collect_alias_free_columns( +fn try_collect_alias_free_columns( exprs: &[(Arc, String)], ) -> Option> { let mut columns = vec![]; From 886e50bd21eaa7f9f05aa857e18c2054f0b98e23 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 29 Mar 2024 15:55:55 +0300 Subject: [PATCH 46/85] Update optimize_projections.rs --- .../optimize_projections.rs | 106 +++++------------- 1 file changed, 31 insertions(+), 75 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index a10016e0fe69..da6dc204e38a 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -1527,7 +1527,7 @@ impl ProjectionOptimizer { let new_filter = rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); - let new_projection = update_hj_projection( + let new_projection = update_hj_projection_right( hj.projection.clone(), hj.left().schema(), hj_left_requirements, @@ -1570,7 +1570,7 @@ impl ProjectionOptimizer { 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( + let new_projection = update_hj_projection_right( hj.projection.clone(), hj.left().schema(), hj_left_requirements, @@ -1614,7 +1614,7 @@ impl ProjectionOptimizer { 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( + let new_projection = update_hj_projection_right( hj.projection.clone(), hj.left().schema(), hj_left_requirements, @@ -5014,6 +5014,28 @@ fn update_hj_projection( }) } +fn update_hj_projection_right( + projection: Option>, + hj_left_schema: SchemaRef, + hj_left_requirements: HashSet, + left_mapping: HashMap, + right_mapping: HashMap, + join_left_input_size: usize, +) -> 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. /// /// This function updates the column indices in a filter's predicate based on a provided mapping. @@ -6527,24 +6549,6 @@ mod tests { #[test] fn test_optimize_projections_filter_sort() -> Result<()> { - /* - INITIAL PLAN: - FilterExec(sum > 0): |sum@0 | - ProjectionExec: |c@2+x@0 as sum | - ProjectionExec: |x@2 |x@0 |c@1 | - SortExec(c@1, x@2): |x@0 |c@1 |x@2 | - ProjectionExec: |x@1 |c@0 |a@2 as x | - ProjectionExec: |c@2 |e@4 as x |a@0 | - CsvExec: |a |b |c |d |e | - ============================================================================================================= - OPTIMIZED PLAN: - FilterExec(sum > 0): |sum@0 | - ProjectionExec: |c@0+x@1 as sum | - SortExec(c@0, x@1): |c@0 |x@1 | - ProjectionExec: |c@1 |a@0 as x | - CsvExec: |a |c | - */ - let csv = create_simple_csv_exec(); let projection1 = Arc::new(ProjectionExec::try_new( @@ -6638,30 +6642,6 @@ mod tests { #[test] fn test_optimize_projections_left_anti() -> Result<()> { - /* - INITIAL PLAN: - AggregateExec(gb(a@2), sum(rownumber@1)): |gb(a) |sum(rownumber) | - HashJoin(a@1=b@1, rownumber@4 Result<()> { - /* - INITIAL PLAN: - AggregateExec(gb(a@2), sum(rank@1)): |gb(a) |sum(rank) | - HashJoin(a@1=b@1, rownumber@4 Date: Fri, 29 Mar 2024 16:11:46 +0300 Subject: [PATCH 47/85] Update optimize_projections.rs --- .../optimize_projections.rs | 29 +------------------ 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index da6dc204e38a..c2d2e22cf379 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -79,7 +79,6 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::displayable; use datafusion_physical_plan::insert::FileSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, @@ -97,13 +96,6 @@ use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use itertools::Itertools; -fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) -} - /// The tree node for the rule of [`OptimizeProjections`]. It stores the necessary /// fields for column requirements and changed indices of columns. #[derive(Debug, Clone)] @@ -1477,7 +1469,6 @@ impl ProjectionOptimizer { } } JoinType::RightSemi | JoinType::RightAnti => { - let join_left_input_size = hj.left().schema().fields().len(); let join_projection = hj .projection .clone() @@ -1529,11 +1520,7 @@ impl ProjectionOptimizer { let new_projection = update_hj_projection_right( hj.projection.clone(), - hj.left().schema(), - hj_left_requirements, - left_mapping, right_mapping, - join_left_input_size, ); let new_hash_join = HashJoinExec::try_new( @@ -1572,11 +1559,7 @@ impl ProjectionOptimizer { rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); let new_projection = update_hj_projection_right( hj.projection.clone(), - hj.left().schema(), - hj_left_requirements, left_mapping, - right_mapping, - join_left_input_size, ); let new_hash_join = HashJoinExec::try_new( @@ -1616,11 +1599,7 @@ impl ProjectionOptimizer { rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); let new_projection = update_hj_projection_right( hj.projection.clone(), - hj.left().schema(), - hj_left_requirements, - left_mapping, right_mapping, - join_left_input_size, ); let new_hash_join = HashJoinExec::try_new( @@ -5016,11 +4995,7 @@ fn update_hj_projection( fn update_hj_projection_right( projection: Option>, - hj_left_schema: SchemaRef, - hj_left_requirements: HashSet, - left_mapping: HashMap, right_mapping: HashMap, - join_left_input_size: usize, ) -> Option> { projection.map(|projection| { projection @@ -5543,9 +5518,7 @@ mod tests { rank, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, RowNumber, Sum, }; - use datafusion_physical_expr::window::{ - BuiltInWindowExpr, BuiltInWindowFunctionExpr, - }; + use datafusion_physical_expr::window::BuiltInWindowExpr; use datafusion_physical_expr::{ Partitioning, PhysicalExpr, PhysicalSortExpr, ScalarFunctionExpr, }; From 9ad9834bbfd289134d65eb2b8c21e11b29479d8e Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 13:22:30 +0300 Subject: [PATCH 48/85] Fix projection renaming --- datafusion/core/src/physical_planner.rs | 35 +++-- .../src/equivalence/projection.rs | 1 + datafusion/physical-plan/src/projection.rs | 18 +-- .../test_files/agg_func_substitute.slt | 12 +- .../sqllogictest/test_files/aggregate.slt | 4 +- .../sqllogictest/test_files/distinct_on.slt | 6 +- .../sqllogictest/test_files/group_by.slt | 138 +++++++++--------- datafusion/sqllogictest/test_files/joins.slt | 14 +- 8 files changed, 117 insertions(+), 111 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 0a1730e944d3..c119c3658148 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -87,7 +87,7 @@ use datafusion_expr::expr::{ use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ - DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, + expr_vec_fmt, DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::expressions::Literal; @@ -107,6 +107,7 @@ fn create_function_physical_name( fun: &str, distinct: bool, args: &[Expr], + order_by: Option>, ) -> Result { let names: Vec = args .iter() @@ -117,7 +118,12 @@ fn create_function_physical_name( true => "DISTINCT ", false => "", }; - Ok(format!("{}({}{})", fun, distinct_str, names.join(","))) + + let phys_name = format!("{}({}{})", fun, distinct_str, names.join(",")); + + Ok(order_by + .map(|order_by| format!("{} ORDER BY [{}]", phys_name, expr_vec_fmt!(order_by))) + .unwrap_or(phys_name)) } fn physical_name(e: &Expr) -> Result { @@ -237,11 +243,19 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { return internal_err!("Function `Expr` with name should be resolved."); } - create_function_physical_name(fun.name(), false, &fun.args) - } - Expr::WindowFunction(WindowFunction { fun, args, .. }) => { - create_function_physical_name(&fun.to_string(), false, args) + create_function_physical_name(fun.name(), false, &fun.args, None) } + Expr::WindowFunction(WindowFunction { + fun, + args, + order_by, + .. + }) => create_function_physical_name( + &fun.to_string(), + false, + args, + Some(order_by.clone()), + ), Expr::AggregateFunction(AggregateFunction { func_def, distinct, @@ -250,9 +264,12 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { order_by, null_treatment: _, }) => match func_def { - AggregateFunctionDefinition::BuiltIn(..) => { - create_function_physical_name(func_def.name(), *distinct, args) - } + AggregateFunctionDefinition::BuiltIn(..) => create_function_physical_name( + func_def.name(), + *distinct, + args, + order_by.clone(), + ), AggregateFunctionDefinition::UDF(fun) => { // TODO: Add support for filter and order by in AggregateUDF if filter.is_some() { diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 732310f9ee2f..7bc82587f5de 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -51,6 +51,7 @@ impl ProjectionMapping { expr: Vec<(Arc, String)>, input_schema: &SchemaRef, ) -> Result { + // Construct a map from the input expressions to the output expression of the projection: expr.into_iter() .enumerate() .map(|(expr_idx, (expression, name))| { diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 1ce8fba21cf1..8d204ca5a283 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -65,22 +65,10 @@ pub struct ProjectionExec { impl ProjectionExec { /// Create a projection on an input pub fn try_new( - mut expr: Vec<(Arc, String)>, + expr: Vec<(Arc, String)>, input: Arc, ) -> Result { let input_schema = input.schema(); - let expr_names = expr - .iter() - .map(|(_, alias)| alias.to_string()) - .collect::>(); - // construct a map from the input expressions to the output expression of the Projection - let projection_mapping = ProjectionMapping::try_new(expr, &input_schema)?; - expr = projection_mapping - .map - .iter() - .zip(expr_names) - .map(|((source, _), name)| (source.clone(), name)) - .collect(); let fields: Result> = expr .iter() @@ -93,16 +81,16 @@ impl ProjectionExec { field.set_metadata( get_field_metadata(e, &input_schema).unwrap_or_default(), ); - Ok(field) }) .collect(); - let schema = Arc::new(Schema::new_with_metadata( 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.clone(), &input_schema)?; let cache = Self::compute_properties(&input, &projection_mapping, schema.clone())?; Ok(Self { diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 61148980f2ff..49dfb60fcf9e 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -43,12 +43,12 @@ Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1 --Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] ----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1))@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=8192 --------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted ------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -63,12 +63,12 @@ Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1 --Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] ----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1))@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=8192 --------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1))], ordering_mode=Sorted +----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted ------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 291b6d677561..e00df9ddec75 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -132,9 +132,9 @@ logical_plan Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] --TableScan: agg_order projection=[c1, c2, c3] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] --CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] ------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] --------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index d897d7c733d4..dedde727d869 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -94,13 +94,13 @@ Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 A ----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] ------TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3)@1 as c3, FIRST_VALUE(aggregate_test_100.c2)@2 as c2] +ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] --SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ----SortExec: expr=[c1@0 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] --------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 78dc6e113299..c3c188fe3bac 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2015,11 +2015,11 @@ Sort: l.col0 ASC NULLS LAST physical_plan SortPreservingMergeExec: [col0@0 ASC NULLS LAST] --SortExec: expr=[col0@0 ASC NULLS LAST] -----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1)@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] +------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] --------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2144,8 +2144,8 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotate --Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan -ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c)@2 as summation1] ---AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) +ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] +--AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) ----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III @@ -2177,8 +2177,8 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE( --Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c)@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted +ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted ----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2203,8 +2203,8 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a --Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted +ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted ----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2292,7 +2292,7 @@ logical_plan Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]]] --TableScan: annotated_data_infinite2 projection=[a, b, d] physical_plan -AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_infinite2.d)], ordering_mode=Sorted +AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], ordering_mode=Sorted --PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] ----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] @@ -2463,8 +2463,8 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales --Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2493,8 +2493,8 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] ----SubqueryAlias: s ------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2537,8 +2537,8 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2573,8 +2573,8 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[zip_code, country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount)@2 as amounts, SUM(s.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) +ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] +--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=PartiallySorted([0]) ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2609,8 +2609,8 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2644,8 +2644,8 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, ------Sort: sales_global.country ASC NULLS LAST --------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount)@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2676,8 +2676,8 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales --Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts, FIRST_VALUE(sales_global.amount)@2 as fv1, LAST_VALUE(sales_global.amount)@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2707,8 +2707,8 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales --Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts, FIRST_VALUE(sales_global.amount)@2 as fv1, LAST_VALUE(sales_global.amount)@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2739,8 +2739,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal --Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as fv2, ARRAY_AGG(sales_global.amount)@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2769,8 +2769,8 @@ Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_globa --Aggregate: groupBy=[[sales_global.country]], aggr=[[SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount)@1 as sum1, ARRAY_AGG(sales_global.amount)@2 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ----SortExec: expr=[amount@2 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2804,8 +2804,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ----Sort: sales_global.ts ASC NULLS LAST ------TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, LAST_VALUE(sales_global.amount)@1 as fv1, FIRST_VALUE(sales_global.amount)@2 as lv1, SUM(sales_global.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2837,8 +2837,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal --Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] ----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as lv1, SUM(sales_global.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] +ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2873,8 +2873,8 @@ Sort: s.sn ASC NULLS LAST ------------TableScan: sales_global projection=[sn, ts, currency, amount] physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] ---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)@5 as last_rate] -----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)] +--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] +----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]] ------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] --------CoalesceBatchesExec: target_batch_size=8192 ----------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] @@ -2918,12 +2918,12 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2954,12 +2954,12 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount)@1 as fv1, LAST_VALUE(sales_global.amount)@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2990,10 +2990,10 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS --Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount)@0 as fv1, LAST_VALUE(sales_global.amount)@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] +--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] --------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3016,10 +3016,10 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS --Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount)@0 as fv1, LAST_VALUE(sales_global.amount)@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] +--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] --------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3040,10 +3040,10 @@ Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS L --Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as array_agg1] +--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] --------SortExec: expr=[ts@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3064,10 +3064,10 @@ Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS --Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] ----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@0 as array_agg1] +--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] --------SortExec: expr=[ts@0 DESC] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3088,10 +3088,10 @@ Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NUL --Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@0 as array_agg1] +--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] +------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] --------SortExec: expr=[amount@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3117,11 +3117,11 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as array_agg1] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] +----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] --------------SortExec: expr=[amount@1 ASC NULLS LAST] ----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3153,11 +3153,11 @@ Sort: sales_global.country ASC NULLS LAST physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount)@1 as amounts, FIRST_VALUE(sales_global.amount)@2 as fv1, LAST_VALUE(sales_global.amount)@3 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] --------------SortExec: expr=[amount@1 DESC] ----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3799,11 +3799,11 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab --Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a)@1 as first_a, FIRST_VALUE(multiple_ordered_table.c)@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -3869,8 +3869,8 @@ Projection: amount_usd ----------------SubqueryAlias: r ------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d)@1 as amount_usd] ---AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true @@ -4974,7 +4974,7 @@ Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULL --Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] ----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan -AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.c)], ordering_mode=Sorted +AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]], ordering_mode=Sorted --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query II? diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 10f3654e6c3f..2a7e4942884f 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3359,8 +3359,8 @@ Sort: l.a ASC NULLS FIRST --------SubqueryAlias: r ----------TableScan: annotated_data projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b)@3 as last_col1] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -3407,8 +3407,8 @@ Projection: amount_usd ----------------SubqueryAlias: r ------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d)@1 as amount_usd] ---AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 ------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true @@ -3444,11 +3444,11 @@ Sort: l.a ASC NULLS FIRST physical_plan SortPreservingMergeExec: [a@0 ASC] --SortExec: expr=[a@0 ASC] -----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b)@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] --------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] --------------CoalesceBatchesExec: target_batch_size=2 ----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] ------------------CoalesceBatchesExec: target_batch_size=2 From 28fcbdca768aab67c1ed99f04e2decabe5334d89 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 16:09:35 +0300 Subject: [PATCH 49/85] agg fixes --- datafusion/core/src/physical_planner.rs | 16 ++++----- .../functions-aggregate/src/first_last.rs | 33 +++++++++++++++++-- 2 files changed, 38 insertions(+), 11 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 3ab8982c7744..89a653c05335 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -85,6 +85,7 @@ 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::{ expr_vec_fmt, DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, @@ -107,7 +108,7 @@ fn create_function_physical_name( fun: &str, distinct: bool, args: &[Expr], - order_by: Option>, + order_by: Option<&Vec>, ) -> Result { let names: Vec = args .iter() @@ -250,25 +251,22 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { args, order_by, .. - }) => create_function_physical_name( - &fun.to_string(), - false, - args, - Some(order_by.clone()), - ), + }) => { + create_function_physical_name(&fun.to_string(), false, args, Some(order_by)) + } Expr::AggregateFunction(AggregateFunction { func_def, distinct, args, filter, - order_by: _, + order_by, null_treatment: _, }) => match func_def { AggregateFunctionDefinition::BuiltIn(..) => create_function_physical_name( func_def.name(), *distinct, args, - order_by.clone(), + order_by.as_ref(), ), AggregateFunctionDefinition::UDF(fun) => { // TODO: Add support for filter by in AggregateUDF diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index d5367ad34163..8c17854a8aae 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -405,11 +405,13 @@ impl FirstValuePhysicalExpr { } pub fn convert_to_last(self) -> LastValuePhysicalExpr { - let name = if self.name.starts_with("FIRST") { + let mut name = if self.name.starts_with("FIRST") { format!("LAST{}", &self.name[5..]) } else { format!("LAST_VALUE({})", self.expr) }; + replace_order_by_clause(&mut name); + let FirstValuePhysicalExpr { expr, input_data_type, @@ -583,11 +585,13 @@ impl LastValuePhysicalExpr { } pub fn convert_to_first(self) -> FirstValuePhysicalExpr { - let name = if self.name.starts_with("LAST") { + let mut name = if self.name.starts_with("LAST") { format!("FIRST{}", &self.name[4..]) } else { format!("FIRST_VALUE({})", self.expr) }; + replace_order_by_clause(&mut name); + let LastValuePhysicalExpr { expr, input_data_type, @@ -895,6 +899,31 @@ fn convert_to_sort_cols( .collect::>() } +fn replace_order_by_clause(order_by: &mut String) { + let suffixes = [ + (" DESC NULLS FIRST]", " ASC NULLS LAST]"), + (" ASC NULLS FIRST]", " DESC NULLS LAST]"), + (" DESC NULLS LAST]", " ASC NULLS FIRST]"), + (" ASC NULLS LAST]", " DESC NULLS FIRST]"), + ]; + + if let Some(start) = order_by.find("ORDER BY [") { + if let Some(end) = order_by[start..].find(']') { + let order_by_start = start + 9; + let order_by_end = start + end; + + let column_order = &order_by[order_by_start..=order_by_end]; + for &(suffix, replacement) in &suffixes { + if column_order.ends_with(suffix) { + let new_order = column_order.replace(suffix, replacement); + order_by.replace_range(order_by_start..=order_by_end, &new_order); + break; + } + } + } + } +} + #[cfg(test)] mod tests { use arrow::array::Int64Array; From 73147b5b66494fe2d0ddf5a8d1db4ad200ca12ac Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 16:18:46 +0300 Subject: [PATCH 50/85] test updates --- .../test_files/agg_func_substitute.slt | 32 +- .../sqllogictest/test_files/aggregate.slt | 12 +- .../sqllogictest/test_files/distinct_on.slt | 18 +- .../sqllogictest/test_files/group_by.slt | 1214 +++++++------- datafusion/sqllogictest/test_files/joins.slt | 1414 ++++++++--------- 5 files changed, 1345 insertions(+), 1345 deletions(-) diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 49dfb60fcf9e..e4d108badaed 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -43,14 +43,14 @@ Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1 --Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] ----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -----SortExec: expr=[a@0 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +03)----SortExec: expr=[a@0 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT @@ -63,14 +63,14 @@ Projection: multiple_ordered_table.a, NTH_VALUE(multiple_ordered_table.c,Int64(1 --Aggregate: groupBy=[[multiple_ordered_table.a]], aggr=[[NTH_VALUE(multiple_ordered_table.c, Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]]] ----TableScan: multiple_ordered_table projection=[a, c] physical_plan -ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -----SortExec: expr=[a@0 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[a@0 as a, NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]@1 as result] +02)--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +03)----SortExec: expr=[a@0 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[NTH_VALUE(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT a, ARRAY_AGG(c ORDER BY c)[1 + 100] as result diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 5f43cef9f3ef..1310e9fad400 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -132,12 +132,12 @@ logical_plan Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] --TableScan: agg_order projection=[c1, c2, c3] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] -------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +01)AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] +04)------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type statement ok diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index dedde727d869..ded6c778dba3 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -94,15 +94,15 @@ Projection: FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 A ----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]]] ------TableScan: aggregate_test_100 projection=[c1, c2, c3] physical_plan -ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] ---SortPreservingMergeExec: [c1@0 ASC NULLS LAST] -----SortExec: expr=[c1@0 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] +02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +03)----SortExec: expr=[c1@0 ASC NULLS LAST] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # ON expressions are not a sub-set of the ORDER BY expressions query error SELECT DISTINCT ON expressions must match initial ORDER BY expressions diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index c3c188fe3bac..5421c3a57dde 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2004,31 +2004,31 @@ GROUP BY l.col0, l.col1, l.col2 ORDER BY l.col0; ---- logical_plan -Sort: l.col0 ASC NULLS LAST ---Projection: l.col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST] AS last_col1 -----Aggregate: groupBy=[[l.col0, l.col1, l.col2]], aggr=[[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]]] -------Inner Join: l.col0 = r.col0 ---------SubqueryAlias: l -----------TableScan: tab0 projection=[col0, col1, col2] ---------SubqueryAlias: r -----------TableScan: tab0 projection=[col0, col1] +01)Sort: l.col0 ASC NULLS LAST +02)--Projection: l.col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST] AS last_col1 +03)----Aggregate: groupBy=[[l.col0, l.col1, l.col2]], aggr=[[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]]] +04)------Inner Join: l.col0 = r.col0 +05)--------SubqueryAlias: l +06)----------TableScan: tab0 projection=[col0, col1, col2] +07)--------SubqueryAlias: r +08)----------TableScan: tab0 projection=[col0, col1] physical_plan -SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ---SortExec: expr=[col0@0 ASC NULLS LAST] -----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] ---------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] +01)SortPreservingMergeExec: [col0@0 ASC NULLS LAST] +02)--SortExec: expr=[col0@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] +04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] +08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +13)------------------------MemoryExec: partitions=1, partition_sizes=[3] +14)--------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +16)------------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2067,13 +2067,13 @@ query TT EXPLAIN SELECT a FROM multiple_ordered_table ORDER BY b DESC; ---- logical_plan -Projection: multiple_ordered_table.a ---Sort: multiple_ordered_table.b DESC NULLS FIRST -----TableScan: multiple_ordered_table projection=[a, b] +01)Projection: multiple_ordered_table.a +02)--Sort: multiple_ordered_table.b DESC NULLS FIRST +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a] ---SortExec: expr=[b@1 DESC] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a] +02)--SortExec: expr=[b@1 DESC] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # Final plan shouldn't have SortExec c ASC, # because table already satisfies this ordering. @@ -2081,9 +2081,9 @@ query TT EXPLAIN SELECT a FROM multiple_ordered_table ORDER BY c ASC; ---- logical_plan -Projection: multiple_ordered_table.a ---Sort: multiple_ordered_table.c ASC NULLS LAST -----TableScan: multiple_ordered_table projection=[a, c] +01)Projection: multiple_ordered_table.a +02)--Sort: multiple_ordered_table.c ASC NULLS LAST +03)----TableScan: multiple_ordered_table projection=[a, c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true # Final plan shouldn't have SortExec a ASC, b ASC, @@ -2092,9 +2092,9 @@ query TT EXPLAIN SELECT a FROM multiple_ordered_table ORDER BY a ASC, b ASC; ---- logical_plan -Projection: multiple_ordered_table.a ---Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST -----TableScan: multiple_ordered_table projection=[a, b] +01)Projection: multiple_ordered_table.a +02)--Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +03)----TableScan: multiple_ordered_table projection=[a, b] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC NULLS LAST], has_header=true # test_window_agg_sort @@ -2109,13 +2109,13 @@ EXPLAIN SELECT a, b, GROUP BY b, a ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotated_data_infinite2.c) AS summation1 ---Aggregate: groupBy=[[annotated_data_infinite2.b, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64))]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotated_data_infinite2.c) AS summation1 +02)--Aggregate: groupBy=[[annotated_data_infinite2.b, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64))]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] ---AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] +02)--AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2140,13 +2140,13 @@ EXPLAIN SELECT a, d, GROUP BY d, a ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS summation1 ---Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] -----TableScan: annotated_data_infinite2 projection=[a, c, d] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS summation1 +02)--Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +03)----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan -ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] ---AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) -----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] +01)ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] +02)--AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=PartiallySorted([1]) +03)----StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST] query III SELECT a, d, @@ -2173,13 +2173,13 @@ EXPLAIN SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c +02)--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c @@ -2199,13 +2199,13 @@ EXPLAIN SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c +02)--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, LAST_VALUE(c ORDER BY a DESC, c ASC) as last_c @@ -2226,13 +2226,13 @@ EXPLAIN SELECT a, b, LAST_VALUE(c) as last_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) AS last_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c)]] -----TableScan: annotated_data_infinite2 projection=[a, b, c] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) AS last_c +02)--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c)]] +03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted -----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III SELECT a, b, LAST_VALUE(c) as last_c @@ -2250,11 +2250,11 @@ FROM annotated_data_infinite2 ORDER BY a, b, d; ---- logical_plan -Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST ---TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] +01)Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST +02)--TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -PartialSortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] ---StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +01)PartialSortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +02)--StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT EXPLAIN SELECT * @@ -2263,13 +2263,13 @@ ORDER BY a, b, d LIMIT 50; ---- logical_plan -Limit: skip=0, fetch=50 ---Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST, fetch=50 -----TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] +01)Limit: skip=0, fetch=50 +02)--Sort: annotated_data_infinite2.a ASC NULLS LAST, annotated_data_infinite2.b ASC NULLS LAST, annotated_data_infinite2.d ASC NULLS LAST, fetch=50 +03)----TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -GlobalLimitExec: skip=0, fetch=50 ---PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] -----StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +01)GlobalLimitExec: skip=0, fetch=50 +02)--PartialSortExec: TopK(fetch=50), expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST], common_prefix_length=[2] +03)----StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] query TT EXPLAIN SELECT * @@ -2277,11 +2277,11 @@ FROM multiple_ordered_table ORDER BY a, b, d; ---- logical_plan -Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST ---TableScan: multiple_ordered_table projection=[a0, a, b, c, d] +01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST +02)--TableScan: multiple_ordered_table projection=[a0, a, b, c, d] physical_plan -SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST] ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +01)SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST] +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY d) @@ -2289,12 +2289,12 @@ FROM annotated_data_infinite2 GROUP BY a, b; ---- logical_plan -Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]]] ---TableScan: annotated_data_infinite2 projection=[a, b, d] +01)Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]]] +02)--TableScan: annotated_data_infinite2 projection=[a, b, d] physical_plan -AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], ordering_mode=Sorted ---PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] -----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] +01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_infinite2.d) ORDER BY [annotated_data_infinite2.d ASC NULLS LAST]], ordering_mode=Sorted +02)--PartialSortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@2 ASC NULLS LAST], common_prefix_length=[2] +03)----StreamingTableExec: partition_sizes=1, projection=[a, b, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST] # as can be seen in the result below d is indeed ordered. query II? @@ -2459,14 +2459,14 @@ EXPLAIN SELECT country, (ARRAY_AGG(amount ORDER BY amount ASC)) AS amounts GROUP BY country ---- logical_plan -Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts ---Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +03)----SortExec: expr=[amount@1 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T? @@ -2488,15 +2488,15 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)] +03)----SortExec: expr=[amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2531,16 +2531,16 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted +03)----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2567,16 +2567,16 @@ EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) GROUP BY s.country, s.zip_code ---- logical_plan -Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[zip_code, country, amount] +01)Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[zip_code, country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=PartiallySorted([0]) -----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=PartiallySorted([0]) +03)----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TI?R rowsort SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2603,16 +2603,16 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted +03)----SortExec: expr=[country@0 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2638,16 +2638,16 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC, s.amount D GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 ---Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] -----SubqueryAlias: s -------Sort: sales_global.country ASC NULLS LAST ---------TableScan: sales_global projection=[country, amount] +01)Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +02)--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(CAST(s.amount AS Float64))]] +03)----SubqueryAlias: s +04)------Sort: sales_global.country ASC NULLS LAST +05)--------TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST], SUM(s.amount)], ordering_mode=Sorted +03)----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC, s.amount DESC) AS amounts, @@ -2672,14 +2672,14 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, GROUP BY country ---- logical_plan -Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +03)----SortExec: expr=[amount@1 DESC] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2703,14 +2703,14 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, GROUP BY country ---- logical_plan -Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +03)----SortExec: expr=[amount@1 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2735,14 +2735,14 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, GROUP BY country ---- logical_plan -Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts ---Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[country, amount] +01)Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +03)----SortExec: expr=[amount@1 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2765,14 +2765,14 @@ EXPLAIN SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, GROUP BY country ---- logical_plan -Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts ---Aggregate: groupBy=[[sales_global.country]], aggr=[[SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[country, ts, amount] +01)Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] -----SortExec: expr=[amount@2 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +03)----SortExec: expr=[amount@2 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2799,14 +2799,14 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, GROUP BY country ---- logical_plan -Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----Sort: sales_global.ts ASC NULLS LAST -------TableScan: sales_global projection=[country, ts, amount] +01)Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----Sort: sales_global.ts ASC NULLS LAST +04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2833,13 +2833,13 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, GROUP BY country ---- logical_plan -Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 ---Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----TableScan: sales_global projection=[country, ts, amount] +01)Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS sum1 +02)--Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(CAST(sales_global.amount AS Float64)) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2862,39 +2862,39 @@ GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency ORDER BY s.sn ---- logical_plan -Sort: s.sn ASC NULLS LAST ---Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST] AS last_rate -----Aggregate: groupBy=[[s.sn, s.zip_code, s.country, s.ts, s.currency]], aggr=[[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]]] -------Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, e.sn, e.amount ---------Inner Join: s.currency = e.currency Filter: s.ts >= e.ts -----------SubqueryAlias: s -------------TableScan: sales_global projection=[zip_code, country, sn, ts, currency] -----------SubqueryAlias: e -------------TableScan: sales_global projection=[sn, ts, currency, amount] +01)Sort: s.sn ASC NULLS LAST +02)--Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST] AS last_rate +03)----Aggregate: groupBy=[[s.sn, s.zip_code, s.country, s.ts, s.currency]], aggr=[[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]]] +04)------Projection: s.zip_code, s.country, s.sn, s.ts, s.currency, e.sn, e.amount +05)--------Inner Join: s.currency = e.currency Filter: s.ts >= e.ts +06)----------SubqueryAlias: s +07)------------TableScan: sales_global projection=[zip_code, country, sn, ts, currency] +08)----------SubqueryAlias: e +09)------------TableScan: sales_global projection=[sn, ts, currency, amount] physical_plan -SortExec: expr=[sn@2 ASC NULLS LAST] ---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] -----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]] -------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] ---------CoalesceBatchesExec: target_batch_size=8192 -----------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] -------------MemoryExec: partitions=1, partition_sizes=[1] -------------MemoryExec: partitions=1, partition_sizes=[1] - -query ITIPTR +01)SortExec: expr=[sn@2 ASC NULLS LAST] +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 +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] + +query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate FROM sales_global AS s JOIN sales_global AS e ON s.currency = e.currency AND s.ts >= e.ts GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency -ORDER BY s.sn, s.zip_code +ORDER BY s.sn ---- 0 GRC 0 2022-01-01T06:00:00 EUR 30 +0 GRC 4 2022-01-03T10:00:00 EUR 80 1 FRA 1 2022-01-01T08:00:00 EUR 50 -1 TUR 2 2022-01-01T11:30:00 TRY 75 1 FRA 3 2022-01-02T12:00:00 EUR 200 -0 GRC 4 2022-01-03T10:00:00 EUR 80 +1 TUR 2 2022-01-01T11:30:00 TRY 75 1 TUR 4 2022-01-03T10:00:00 TRY 100 # Run order-sensitive aggregators in multiple partitions @@ -2911,20 +2911,20 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] -------TableScan: sales_global projection=[country, ts, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] +04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2947,20 +2947,20 @@ EXPLAIN SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -------TableScan: sales_global projection=[country, ts, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -----------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2986,16 +2986,16 @@ EXPLAIN SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, FROM sales_global ---- logical_plan -Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 ---Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv2 +02)--Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3012,16 +3012,16 @@ EXPLAIN SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, FROM sales_global ---- logical_plan -Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 ---Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS fv2 +02)--Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -3036,17 +3036,17 @@ EXPLAIN SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 FROM sales_global ---- logical_plan -Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST] AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +05)--------SortExec: expr=[ts@0 ASC NULLS LAST] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -3060,17 +3060,17 @@ EXPLAIN SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 FROM sales_global ---- logical_plan -Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] -----TableScan: sales_global projection=[ts, amount] +01)Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST] AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]]] +03)----TableScan: sales_global projection=[ts, amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] ---------SortExec: expr=[ts@0 DESC] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +05)--------SortExec: expr=[ts@0 DESC] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -3084,17 +3084,17 @@ EXPLAIN SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 FROM sales_global ---- logical_plan -Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -----TableScan: sales_global projection=[amount] +01)Projection: ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +03)----TableScan: sales_global projection=[amount] physical_plan -ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ---------SortExec: expr=[amount@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +05)--------SortExec: expr=[amount@0 ASC NULLS LAST] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3110,21 +3110,21 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] -------TableScan: sales_global projection=[country, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS array_agg1 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] +04)------TableScan: sales_global projection=[country, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] ---------------SortExec: expr=[amount@1 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as array_agg1] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +08)--------------SortExec: expr=[amount@1 ASC NULLS LAST] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------MemoryExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3146,21 +3146,21 @@ EXPLAIN SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, ORDER BY country ---- logical_plan -Sort: sales_global.country ASC NULLS LAST ---Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 -----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] -------TableScan: sales_global projection=[country, amount] +01)Sort: sales_global.country ASC NULLS LAST +02)--Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST] AS fv2 +03)----Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]]] +04)------TableScan: sales_global projection=[country, amount] physical_plan -SortPreservingMergeExec: [country@0 ASC NULLS LAST] ---SortExec: expr=[country@0 ASC NULLS LAST] -----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] ---------------SortExec: expr=[amount@1 DESC] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] +02)--SortExec: expr=[country@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +08)--------------SortExec: expr=[amount@1 DESC] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3346,20 +3346,20 @@ EXPLAIN SELECT s.sn, s.amount, 2*s.sn ORDER BY sn ---- logical_plan -Sort: s.sn ASC NULLS LAST ---Projection: s.sn, s.amount, Int64(2) * CAST(s.sn AS Int64) -----Aggregate: groupBy=[[s.sn, s.amount]], aggr=[[]] -------SubqueryAlias: s ---------TableScan: sales_global_with_pk projection=[sn, amount] +01)Sort: s.sn ASC NULLS LAST +02)--Projection: s.sn, s.amount, Int64(2) * CAST(s.sn AS Int64) +03)----Aggregate: groupBy=[[s.sn, s.amount]], aggr=[[]] +04)------SubqueryAlias: s +05)--------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -SortPreservingMergeExec: [sn@0 ASC NULLS LAST] ---SortExec: expr=[sn@0 ASC NULLS LAST] -----ProjectionExec: expr=[sn@0 as sn, amount@1 as amount, 2 * CAST(sn@0 AS Int64) as Int64(2) * s.sn] -------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] ---------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +02)--SortExec: expr=[sn@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[sn@0 as sn, amount@1 as amount, 2 * CAST(sn@0 AS Int64) as Int64(2) * s.sn] +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] +08)--------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] query IRI SELECT s.sn, s.amount, 2*s.sn @@ -3410,28 +3410,28 @@ EXPLAIN SELECT r.sn, SUM(l.amount), r.amount ORDER BY r.sn ---- logical_plan -Sort: r.sn ASC NULLS LAST ---Projection: r.sn, SUM(l.amount), r.amount -----Aggregate: groupBy=[[r.sn, r.amount]], aggr=[[SUM(CAST(l.amount AS Float64))]] -------Projection: l.amount, r.sn, r.amount ---------Inner Join: Filter: l.sn >= r.sn -----------SubqueryAlias: l -------------TableScan: sales_global_with_pk projection=[sn, amount] -----------SubqueryAlias: r -------------TableScan: sales_global_with_pk projection=[sn, amount] +01)Sort: r.sn ASC NULLS LAST +02)--Projection: r.sn, SUM(l.amount), r.amount +03)----Aggregate: groupBy=[[r.sn, r.amount]], aggr=[[SUM(CAST(l.amount AS Float64))]] +04)------Projection: l.amount, r.sn, r.amount +05)--------Inner Join: Filter: l.sn >= r.sn +06)----------SubqueryAlias: l +07)------------TableScan: sales_global_with_pk projection=[sn, amount] +08)----------SubqueryAlias: r +09)------------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] ---SortPreservingMergeExec: [sn@0 ASC NULLS LAST] -----SortExec: expr=[sn@0 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[SUM(l.amount)] ---------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] -----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 -------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] -------------------CoalescePartitionsExec ---------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +02)--SortExec: expr=[sn@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[sn@1 as sn, amount@2 as amount], aggr=[SUM(l.amount)] +08)--------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] +09)----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 +10)------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +11)------------------CoalescePartitionsExec +12)--------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] query IRR SELECT r.sn, SUM(l.amount), r.amount @@ -3559,27 +3559,27 @@ EXPLAIN SELECT * ORDER BY l.sn ---- logical_plan -Sort: l.sn ASC NULLS LAST ---Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, l.sum_amount -----Aggregate: groupBy=[[l.sn, l.zip_code, l.country, l.ts, l.currency, l.amount, l.sum_amount]], aggr=[[]] -------SubqueryAlias: l ---------Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum_amount -----------WindowAggr: windowExpr=[[SUM(CAST(l.amount AS Float64)) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -------------SubqueryAlias: l ---------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] +01)Sort: l.sn ASC NULLS LAST +02)--Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, l.sum_amount +03)----Aggregate: groupBy=[[l.sn, l.zip_code, l.country, l.ts, l.currency, l.amount, l.sum_amount]], aggr=[[]] +04)------SubqueryAlias: l +05)--------Projection: l.zip_code, l.country, l.sn, l.ts, l.currency, l.amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS sum_amount +06)----------WindowAggr: windowExpr=[[SUM(CAST(l.amount AS Float64)) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +07)------------SubqueryAlias: l +08)--------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] physical_plan -ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] ---SortPreservingMergeExec: [sn@0 ASC NULLS LAST] -----SortExec: expr=[sn@0 ASC NULLS LAST] -------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] ---------CoalesceBatchesExec: target_batch_size=4 -----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] -------------------BoundedWindowAggExec: wdw=[SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------------CoalescePartitionsExec -----------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +01)SortPreservingMergeExec: [sn@2 ASC NULLS LAST] +02)--SortExec: expr=[sn@2 ASC NULLS LAST] +03)----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] +04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] +05)--------CoalesceBatchesExec: target_batch_size=4 +06)----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] +10)------------------BoundedWindowAggExec: wdw=[SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +11)--------------------CoalescePartitionsExec +12)----------------------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] query ITIPTRR @@ -3755,14 +3755,14 @@ EXPLAIN SELECT LAST_VALUE(x) FROM FOO; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(foo.x)]] ---TableScan: foo projection=[x] +01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(foo.x)]] +02)--TableScan: foo projection=[x] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(foo.x)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(foo.x)] -------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(foo.x)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(foo.x)] +04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] query I SELECT FIRST_VALUE(x) @@ -3777,14 +3777,14 @@ EXPLAIN SELECT FIRST_VALUE(x) FROM FOO; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(foo.x)]] ---TableScan: foo projection=[x] +01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(foo.x)]] +02)--TableScan: foo projection=[x] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(foo.x)] -------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(foo.x)] +04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] # Since both ordering requirements are satisfied, there shouldn't be # any SortExec in the final plan. @@ -3795,17 +3795,17 @@ FROM multiple_ordered_table GROUP BY d; ---- logical_plan -Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST] AS first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] AS last_c ---Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] -----TableScan: multiple_ordered_table projection=[a, c, d] +01)Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST] AS first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] AS last_c +02)--Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] +03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -3837,8 +3837,8 @@ FROM multiple_ordered_table ORDER BY c ASC; ---- logical_plan -Sort: multiple_ordered_table.c ASC NULLS LAST ---TableScan: multiple_ordered_table projection=[c] +01)Sort: multiple_ordered_table.c ASC NULLS LAST +02)--TableScan: multiple_ordered_table projection=[c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true statement ok @@ -3856,27 +3856,27 @@ GROUP BY row_n ORDER BY row_n ---- logical_plan -Projection: amount_usd ---Sort: row_n ASC NULLS LAST -----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n -------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] ---------Projection: l.a, l.d, row_n -----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) -------------SubqueryAlias: l ---------------TableScan: multiple_ordered_table projection=[a, d] -------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n ---------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------------SubqueryAlias: r -------------------TableScan: multiple_ordered_table projection=[a, d] +01)Projection: amount_usd +02)--Sort: row_n ASC NULLS LAST +03)----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +04)------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +05)--------Projection: l.a, l.d, row_n +06)----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +07)------------SubqueryAlias: l +08)--------------TableScan: multiple_ordered_table projection=[a, d] +09)------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +10)--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +11)----------------SubqueryAlias: r +12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] ---AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true ---------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # reset partition number to 8. statement ok @@ -3908,16 +3908,16 @@ FROM multiple_ordered_table_with_pk GROUP BY c; ---- logical_plan -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))]] ---TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)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))]] +02)--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ---SortExec: expr=[c@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +02)--SortExec: expr=[c@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true # drop table multiple_ordered_table_with_pk statement ok @@ -3949,16 +3949,16 @@ FROM multiple_ordered_table_with_pk GROUP BY c; ---- logical_plan -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))]] ---TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)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))]] +02)--TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ---SortExec: expr=[c@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)AggregateExec: mode=FinalPartitioned, gby=[c@0 as c, b@1 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +02)--SortExec: expr=[c@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -3972,15 +3972,15 @@ EXPLAIN SELECT c, sum1 GROUP BY c; ---- logical_plan -Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, sum1]], aggr=[[]] ---Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[c, d] +01)Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, sum1]], aggr=[[]] +02)--Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[c, d] physical_plan -AggregateExec: mode=Single, gby=[c@0 as c, sum1@1 as sum1], aggr=[], ordering_mode=PartiallySorted([0]) ---ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] -----AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)AggregateExec: mode=Single, gby=[c@0 as c, sum1@1 as sum1], aggr=[], ordering_mode=PartiallySorted([0]) +02)--ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +03)----AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb @@ -3990,17 +3990,17 @@ EXPLAIN SELECT c, sum1, SUM(b) OVER() as sumb GROUP BY c); ---- logical_plan -Projection: multiple_ordered_table_with_pk.c, sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sumb ---WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table_with_pk.b AS Int64)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 -------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))]] ---------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)Projection: multiple_ordered_table_with_pk.c, sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sumb +02)--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table_with_pk.b AS Int64)) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +04)------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))]] +05)--------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as sumb] ---WindowAggExec: wdw=[SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED 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(NULL)), is_causal: false }] -----ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[c@0 as c, sum1@2 as sum1, SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as sumb] +02)--WindowAggExec: wdw=[SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(multiple_ordered_table_with_pk.b) ROWS BETWEEN UNBOUNDED 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(NULL)), is_causal: false }] +03)----ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +04)------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4015,26 +4015,26 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 ON lhs.b=rhs.b; ---- logical_plan -Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---Inner Join: lhs.b = rhs.b -----SubqueryAlias: lhs -------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------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))]] -----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] -----SubqueryAlias: rhs -------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------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))]] -----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 +02)--Inner Join: lhs.b = rhs.b +03)----SubqueryAlias: lhs +04)------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +05)--------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))]] +06)----------TableScan: multiple_ordered_table_with_pk projection=[b, c, d] +07)----SubqueryAlias: rhs +08)------Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +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 -ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] -------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true -------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)], projection=[c@0, sum1@2, c@3, sum1@5] +04)------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true +07)------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +08)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) +09)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 @@ -4048,25 +4048,25 @@ EXPLAIN SELECT lhs.c, rhs.c, lhs.sum1, rhs.sum1 GROUP BY c) as rhs; ---- logical_plan -Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 ---CrossJoin: -----SubqueryAlias: lhs -------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----------TableScan: multiple_ordered_table_with_pk projection=[c, d] -----SubqueryAlias: rhs -------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----------TableScan: multiple_ordered_table_with_pk projection=[c, d] +01)Projection: lhs.c, rhs.c, lhs.sum1, rhs.sum1 +02)--CrossJoin: +03)----SubqueryAlias: lhs +04)------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +05)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +06)----------TableScan: multiple_ordered_table_with_pk projection=[c, d] +07)----SubqueryAlias: rhs +08)------Projection: multiple_ordered_table_with_pk.c, SUM(multiple_ordered_table_with_pk.d) AS sum1 +09)--------Aggregate: groupBy=[[multiple_ordered_table_with_pk.c]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +10)----------TableScan: multiple_ordered_table_with_pk projection=[c, d] physical_plan -ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] ---CrossJoinExec -----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] -------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true -----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] -------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[c@0 as c, c@2 as c, sum1@1 as sum1, sum1@3 as sum1] +02)--CrossJoinExec +03)----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +04)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +06)----ProjectionExec: expr=[c@0 as c, SUM(multiple_ordered_table_with_pk.d)@1 as sum1] +07)------AggregateExec: mode=Single, gby=[c@0 as c], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +08)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true # we do not generate physical plan for Repartition yet (e.g Distribute By queries). query TT @@ -4077,10 +4077,10 @@ FROM (SELECT c, b, a, SUM(d) as sum1 DISTRIBUTE BY a ---- logical_plan -Repartition: DistributeBy(a) ---Projection: multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, b, c, d] +01)Repartition: DistributeBy(a) +02)--Projection: multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b, SUM(multiple_ordered_table_with_pk.d) AS sum1 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, multiple_ordered_table_with_pk.b]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[a, b, c, d] # union with aggregate query TT @@ -4093,21 +4093,21 @@ UNION ALL GROUP BY c ---- logical_plan -Union ---Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] ---Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +01)Union +02)--Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +05)--Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +06)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +07)------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] physical_plan -UnionExec ---ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true ---ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] -----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)UnionExec +02)--ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +03)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +05)--ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +06)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +07)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true # table scan should be simplified. query TT @@ -4116,13 +4116,13 @@ EXPLAIN SELECT c, a, SUM(d) as sum1 GROUP BY c ---- logical_plan -Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -----TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +01)Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +02)--Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +03)----TableScan: multiple_ordered_table_with_pk projection=[a, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +02)--AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true # limit should be simplified query TT @@ -4133,15 +4133,15 @@ EXPLAIN SELECT * LIMIT 5) ---- logical_plan -Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 ---Limit: skip=0, fetch=5 -----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] -------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] +01)Projection: multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a, SUM(multiple_ordered_table_with_pk.d) AS sum1 +02)--Limit: skip=0, fetch=5 +03)----Aggregate: groupBy=[[multiple_ordered_table_with_pk.c, multiple_ordered_table_with_pk.a]], aggr=[[SUM(CAST(multiple_ordered_table_with_pk.d AS Int64))]] +04)------TableScan: multiple_ordered_table_with_pk projection=[a, c, d] physical_plan -ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] ---GlobalLimitExec: skip=0, fetch=5 -----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[c@0 as c, a@1 as a, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----AggregateExec: mode=Single, gby=[c@1 as c, a@0 as a], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=Sorted +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 8; @@ -4169,40 +4169,40 @@ query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT x) FROM t1 GROUP BY y; ---- logical_plan -Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) ---Aggregate: groupBy=[[t1.y]], aggr=[[SUM(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] -----TableScan: t1 projection=[x, y] +01)Projection: SUM(DISTINCT t1.x), MAX(DISTINCT t1.x) +02)--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(DISTINCT CAST(t1.x AS Float64)), MAX(DISTINCT t1.x)]] +03)----TableScan: t1 projection=[x, y] physical_plan -ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] ---AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[SUM(DISTINCT t1.x)@1 as SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)@2 as MAX(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[SUM(DISTINCT t1.x), MAX(DISTINCT t1.x)] +07)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN SELECT SUM(DISTINCT CAST(x AS DOUBLE)), MAX(DISTINCT CAST(x AS DOUBLE)) FROM t1 GROUP BY y; ---- logical_plan -Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) ---Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] -----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] -------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y ---------TableScan: t1 projection=[x, y] +01)Projection: SUM(alias1) AS SUM(DISTINCT t1.x), MAX(alias1) AS MAX(DISTINCT t1.x) +02)--Aggregate: groupBy=[[t1.y]], aggr=[[SUM(alias1), MAX(alias1)]] +03)----Aggregate: groupBy=[[t1.y, CAST(t1.x AS Float64)t1.x AS t1.x AS alias1]], aggr=[[]] +04)------Projection: CAST(t1.x AS Float64) AS CAST(t1.x AS Float64)t1.x, t1.y +05)--------TableScan: t1 projection=[x, y] physical_plan -ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] ---AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] -----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] -------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] ---------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] -----------------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[SUM(alias1)@1 as SUM(DISTINCT t1.x), MAX(alias1)@2 as MAX(DISTINCT t1.x)] +02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[y@0 as y], aggr=[SUM(alias1), MAX(alias1)] +06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, CAST(t1.x AS Float64)t1.x@0 as alias1], aggr=[] +11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as CAST(t1.x AS Float64)t1.x, y@1 as y] +12)----------------------MemoryExec: partitions=1, partition_sizes=[1] # create an unbounded table that contains ordered timestamp. statement ok @@ -4223,21 +4223,21 @@ EXPLAIN SELECT date_bin('15 minutes', ts) as time_chunks LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: time_chunks DESC NULLS FIRST, fetch=5 -----Projection: date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts) AS time_chunks -------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)]], aggr=[[]] ---------TableScan: unbounded_csv_with_timestamps projection=[ts] +01)Limit: skip=0, fetch=5 +02)--Sort: time_chunks DESC NULLS FIRST, fetch=5 +03)----Projection: date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts) AS time_chunks +04)------Aggregate: groupBy=[[date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps.ts) AS date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)]], aggr=[[]] +05)--------TableScan: unbounded_csv_with_timestamps projection=[ts] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 -----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] -------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC -------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [time_chunks@0 DESC], fetch=5 +03)----ProjectionExec: expr=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as time_chunks] +04)------AggregateExec: mode=FinalPartitioned, gby=[date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC +07)------------AggregateExec: mode=Partial, gby=[date_bin(900000000000, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] query P SELECT date_bin('15 minutes', ts) as time_chunks @@ -4278,22 +4278,22 @@ EXPLAIN SELECT extract(month from ts) as months LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: months DESC NULLS FIRST, fetch=5 -----Projection: date_part(Utf8("MONTH"),csv_with_timestamps.ts) AS months -------Aggregate: groupBy=[[date_part(Utf8("MONTH"), csv_with_timestamps.ts)]], aggr=[[]] ---------TableScan: csv_with_timestamps projection=[ts] +01)Limit: skip=0, fetch=5 +02)--Sort: months DESC NULLS FIRST, fetch=5 +03)----Projection: date_part(Utf8("MONTH"),csv_with_timestamps.ts) AS months +04)------Aggregate: groupBy=[[date_part(Utf8("MONTH"), csv_with_timestamps.ts)]], aggr=[[]] +05)--------TableScan: csv_with_timestamps projection=[ts] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [months@0 DESC], fetch=5 -----SortExec: TopK(fetch=5), expr=[months@0 DESC] -------ProjectionExec: expr=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as months] ---------AggregateExec: mode=FinalPartitioned, gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] -----------CoalesceBatchesExec: target_batch_size=2 -------------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 ---------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [months@0 DESC], fetch=5 +03)----SortExec: TopK(fetch=5), expr=[months@0 DESC] +04)------ProjectionExec: expr=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as months] +05)--------AggregateExec: mode=FinalPartitioned, gby=[date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0 as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 +08)--------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false query R SELECT extract(month from ts) as months @@ -4326,16 +4326,16 @@ EXPLAIN SELECT name, date_bin('15 minutes', ts) as time_chunks LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: unbounded_csv_with_timestamps2.name DESC NULLS FIRST, time_chunks DESC NULLS FIRST, fetch=5 -----Projection: unbounded_csv_with_timestamps2.name, date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps2.ts) AS time_chunks -------TableScan: unbounded_csv_with_timestamps2 projection=[name, ts] +01)Limit: skip=0, fetch=5 +02)--Sort: unbounded_csv_with_timestamps2.name DESC NULLS FIRST, time_chunks DESC NULLS FIRST, fetch=5 +03)----Projection: unbounded_csv_with_timestamps2.name, date_bin(IntervalMonthDayNano("900000000000"), unbounded_csv_with_timestamps2.ts) AS time_chunks +04)------TableScan: unbounded_csv_with_timestamps2 projection=[name, ts] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [name@0 DESC,time_chunks@1 DESC], fetch=5 -----ProjectionExec: expr=[name@0 as name, date_bin(900000000000, ts@1) as time_chunks] -------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [name@0 DESC,time_chunks@1 DESC], fetch=5 +03)----ProjectionExec: expr=[name@0 as name, date_bin(900000000000, ts@1) as time_chunks] +04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok drop table t1 @@ -4393,25 +4393,25 @@ query TT EXPLAIN SELECT c1, count(distinct c2), min(distinct c2), sum(c3), max(c4) FROM aggregate_test_100 GROUP BY c1 ORDER BY c1; ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, COUNT(alias1) AS COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1) AS MIN(DISTINCT aggregate_test_100.c2), SUM(alias2) AS SUM(aggregate_test_100.c3), MAX(alias3) AS MAX(aggregate_test_100.c4) -----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)]] -------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[SUM(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, MAX(aggregate_test_100.c4) AS alias3]] ---------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, COUNT(alias1) AS COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1) AS MIN(DISTINCT aggregate_test_100.c2), SUM(alias2) AS SUM(aggregate_test_100.c3), MAX(alias3) AS MAX(aggregate_test_100.c4) +03)----Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)]] +04)------Aggregate: groupBy=[[aggregate_test_100.c1, aggregate_test_100.c2 AS alias1]], aggr=[[SUM(CAST(aggregate_test_100.c3 AS Int64)) AS alias2, MAX(aggregate_test_100.c4) AS alias3]] +05)--------TableScan: aggregate_test_100 projection=[c1, c2, c3, c4] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ---SortExec: expr=[c1@0 ASC NULLS LAST] -----ProjectionExec: expr=[c1@0 as c1, COUNT(alias1)@1 as COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1)@2 as MIN(DISTINCT aggregate_test_100.c2), SUM(alias2)@3 as SUM(aggregate_test_100.c3), MAX(alias3)@4 as MAX(aggregate_test_100.c4)] -------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] ---------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 ---------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] -----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +02)--SortExec: expr=[c1@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c1@0 as c1, COUNT(alias1)@1 as COUNT(DISTINCT aggregate_test_100.c2), MIN(alias1)@2 as MIN(DISTINCT aggregate_test_100.c2), SUM(alias2)@3 as SUM(aggregate_test_100.c3), MAX(alias3)@4 as MAX(aggregate_test_100.c4)] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[COUNT(alias1), MIN(alias1), SUM(alias2), MAX(alias3)] +08)--------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1, alias1@1 as alias1], aggr=[alias2, alias3] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 +11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true # Use PostgreSQL dialect statement ok @@ -4573,20 +4573,20 @@ ORDER BY MAX(t1) DESC LIMIT 4; ---- logical_plan -Limit: skip=0, fetch=4 ---Sort: MAX(timestamp_table.t1) DESC NULLS FIRST, fetch=4 -----Aggregate: groupBy=[[timestamp_table.c2]], aggr=[[MAX(timestamp_table.t1)]] -------TableScan: timestamp_table projection=[t1, c2] +01)Limit: skip=0, fetch=4 +02)--Sort: MAX(timestamp_table.t1) DESC NULLS FIRST, fetch=4 +03)----Aggregate: groupBy=[[timestamp_table.c2]], aggr=[[MAX(timestamp_table.t1)]] +04)------TableScan: timestamp_table projection=[t1, c2] physical_plan -GlobalLimitExec: skip=0, fetch=4 ---SortPreservingMergeExec: [MAX(timestamp_table.t1)@1 DESC], fetch=4 -----SortExec: TopK(fetch=4), expr=[MAX(timestamp_table.t1)@1 DESC] -------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 -----------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true +01)GlobalLimitExec: skip=0, fetch=4 +02)--SortPreservingMergeExec: [MAX(timestamp_table.t1)@1 DESC], fetch=4 +03)----SortExec: TopK(fetch=4), expr=[MAX(timestamp_table.t1)@1 DESC] +04)------AggregateExec: mode=FinalPartitioned, gby=[c2@0 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 +07)------------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[MAX(timestamp_table.t1)], lim=[4] +08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 +09)----------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true # Clean up statement ok @@ -4970,12 +4970,12 @@ GROUP BY a, b ORDER BY a, b; ---- logical_plan -Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST ---Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] -----TableScan: multiple_ordered_table projection=[a, b, c] +01)Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +02)--Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] +03)----TableScan: multiple_ordered_table projection=[a, b, c] physical_plan -AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]], ordering_mode=Sorted ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]], ordering_mode=Sorted +02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query II? SELECT a, b, ARRAY_AGG(c ORDER BY c DESC) diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 9a8ae484bdc5..d96e58ddf20c 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1037,11 +1037,11 @@ LEFT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t2.t2_id < 100 ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_id < UInt32(100) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_id < UInt32(100) -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_id < UInt32(100) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--Filter: join_t2.t2_id < UInt32(100) +05)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce left join 2 (to inner join) @@ -1053,10 +1053,10 @@ LEFT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t2.t2_int < 10 or (join_t1.t1_int > 2 and join_t2.t2_name != 'w') ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int < UInt32(10) OR join_t1.t1_int > UInt32(2) AND join_t2.t2_name != Utf8("w") ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_int < UInt32(10) OR join_t2.t2_name != Utf8("w") -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int < UInt32(10) OR join_t1.t1_int > UInt32(2) AND join_t2.t2_name != Utf8("w") +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--Filter: join_t2.t2_int < UInt32(10) OR join_t2.t2_name != Utf8("w") +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce left join 3 (to inner join) @@ -1073,16 +1073,16 @@ LEFT JOIN join_t2 on t3.t1_int = join_t2.t2_int WHERE t3.t1_id < 100 ---- logical_plan -Left Join: t3.t1_int = join_t2.t2_int ---SubqueryAlias: t3 -----Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int -------Inner Join: join_t1.t1_id = join_t2.t2_id ---------Filter: join_t1.t1_id < UInt32(100) -----------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---------Projection: join_t2.t2_id -----------Filter: join_t2.t2_int < UInt32(3) AND join_t2.t2_id < UInt32(100) -------------TableScan: join_t2 projection=[t2_id, t2_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Left Join: t3.t1_int = join_t2.t2_int +02)--SubqueryAlias: t3 +03)----Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int +04)------Inner Join: join_t1.t1_id = join_t2.t2_id +05)--------Filter: join_t1.t1_id < UInt32(100) +06)----------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +07)--------Projection: join_t2.t2_id +08)----------Filter: join_t2.t2_int < UInt32(3) AND join_t2.t2_id < UInt32(100) +09)------------TableScan: join_t2 projection=[t2_id, t2_int] +10)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce right join 1 (to inner join) @@ -1094,10 +1094,10 @@ RIGHT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t1.t1_int IS NOT NULL ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_int IS NOT NULL -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_int IS NOT NULL +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce right join 2 (to inner join) @@ -1109,9 +1109,9 @@ RIGHT JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE NOT (join_t1.t1_int = join_t2.t2_int) ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int != join_t1.t1_int ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id Filter: join_t2.t2_int != join_t1.t1_int +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to right join @@ -1123,10 +1123,10 @@ FULL JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t2.t2_name IS NOT NULL ---- logical_plan -Right Join: join_t1.t1_id = join_t2.t2_id ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_name IS NOT NULL -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Right Join: join_t1.t1_id = join_t2.t2_id +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--Filter: join_t2.t2_name IS NOT NULL +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to left join @@ -1138,10 +1138,10 @@ FULL JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id WHERE join_t1.t1_name != 'b' ---- logical_plan -Left Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_name != Utf8("b") -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Left Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_name != Utf8("b") +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce full join to inner join @@ -1153,11 +1153,11 @@ FULL JOIN join_t2 on join_t1.t1_id = join_t2.t2_id WHERE join_t1.t1_name != 'b' and join_t2.t2_name = 'x' ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id ---Filter: join_t1.t1_name != Utf8("b") -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---Filter: join_t2.t2_name = Utf8("x") -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--Filter: join_t1.t1_name != Utf8("b") +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--Filter: join_t2.t2_name = Utf8("x") +05)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] ### # Configuration teardown @@ -1205,11 +1205,11 @@ FROM lsaj_t1 LEFT SEMI JOIN lsaj_t2 ON (lsaj_t1.t1_id = lsaj_t2.t2_id and lsaj_t2.t2_int > 1) ---- logical_plan -LeftSemi Join: lsaj_t1.t1_id = lsaj_t2.t2_id ---TableScan: lsaj_t1 projection=[t1_id, t1_name] ---Projection: lsaj_t2.t2_id -----Filter: lsaj_t2.t2_int > UInt32(1) -------TableScan: lsaj_t2 projection=[t2_id, t2_int] +01)LeftSemi Join: lsaj_t1.t1_id = lsaj_t2.t2_id +02)--TableScan: lsaj_t1 projection=[t1_id, t1_name] +03)--Projection: lsaj_t2.t2_id +04)----Filter: lsaj_t2.t2_int > UInt32(1) +05)------TableScan: lsaj_t2 projection=[t2_id, t2_int] # Left anti join @@ -1318,23 +1318,23 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] ---Projection: join_t1.t1_id -----Inner Join: join_t1.t1_id = join_t2.t2_id -------TableScan: join_t1 projection=[t1_id] -------TableScan: join_t2 projection=[t2_id] +01)Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[]] +02)--Projection: join_t1.t1_id +03)----Inner Join: join_t1.t1_id = join_t2.t2_id +04)------TableScan: join_t1 projection=[t1_id] +05)------TableScan: join_t2 projection=[t2_id] physical_plan -AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1343,25 +1343,25 @@ from (select * from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id) group by t1_id ---- logical_plan -Projection: COUNT(*) ---Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----Projection: join_t1.t1_id -------Inner Join: join_t1.t1_id = join_t2.t2_id ---------TableScan: join_t1 projection=[t1_id] ---------TableScan: join_t2 projection=[t2_id] +01)Projection: COUNT(*) +02)--Aggregate: groupBy=[[join_t1.t1_id]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----Projection: join_t1.t1_id +04)------Inner Join: join_t1.t1_id = join_t2.t2_id +05)--------TableScan: join_t1 projection=[t1_id] +06)--------TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] ---AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] +02)--AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1370,30 +1370,30 @@ from join_t1 inner join join_t2 on join_t1.t1_id = join_t2.t2_id ---- logical_plan -Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) ---Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] -----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] -------Projection: join_t1.t1_id ---------Inner Join: join_t1.t1_id = join_t2.t2_id -----------TableScan: join_t1 projection=[t1_id] -----------TableScan: join_t2 projection=[t2_id] +01)Projection: COUNT(alias1) AS COUNT(DISTINCT join_t1.t1_id) +02)--Aggregate: groupBy=[[]], aggr=[[COUNT(alias1)]] +03)----Aggregate: groupBy=[[join_t1.t1_id AS alias1]], aggr=[[]] +04)------Projection: join_t1.t1_id +05)--------Inner Join: join_t1.t1_id = join_t2.t2_id +06)----------TableScan: join_t1 projection=[t1_id] +07)----------TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] ---AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] ---------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] -----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -------------CoalesceBatchesExec: target_batch_size=2 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[1] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(alias1)@0 as COUNT(DISTINCT join_t1.t1_id)] +02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(alias1)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] +05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] +06)----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +13)----------------CoalesceBatchesExec: target_batch_size=2 +14)------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +15)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)----------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1408,9 +1408,9 @@ cross join join_t2 where join_t1.t1_id + 12 = join_t2.t2_id + 1 ---- logical_plan -Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = CAST(join_t2.t2_id AS Int64) + Int64(1) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = CAST(join_t2.t2_id AS Int64) + Int64(1) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] # Reduce cross join with cast expr join key (to inner join) @@ -1421,10 +1421,10 @@ from join_t1 cross join join_t2 where join_t1.t1_id + 11 = cast(join_t2.t2_id as BIGINT) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] ##### @@ -1446,21 +1446,21 @@ from join_t1, join_t2 where join_t1.t1_id + 11 = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) ---Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) +02)--Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +04)------CoalescePartitionsExec +05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1472,24 +1472,24 @@ from join_t1, join_t2 where join_t1.t1_id + 11 = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) ---Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Projection: join_t1.t1_id, join_t1.t1_name, join_t1.t1_int, join_t2.t2_id, join_t2.t2_name, join_t2.t2_int, CAST(join_t1.t1_id AS Int64) + Int64(11) +02)--Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 +06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 +11)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] +12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)--------------MemoryExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1503,21 +1503,21 @@ FROM join_t1 INNER JOIN join_t2 ON join_t1.t1_id + cast(12 as INT UNSIGNED) = join_t2.t2_id + cast(1 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalescePartitionsExec +05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1529,24 +1529,24 @@ FROM join_t1 INNER JOIN join_t2 ON join_t1.t1_id + cast(12 as INT UNSIGNED) = join_t2.t2_id + cast(1 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(12) = join_t2.t2_id + UInt32(1) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 +06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 +11)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] +12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)--------------MemoryExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1561,18 +1561,18 @@ INNER JOIN join_t2 ON join_t1.t1_id + cast(11 as INT UNSIGNED) = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -------MemoryExec: partitions=1, partition_sizes=[1] -------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1585,23 +1585,23 @@ INNER JOIN join_t2 ON join_t1.t1_id + cast(11 as INT UNSIGNED) = join_t2.t2_id ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id + UInt32(11) = join_t2.t2_id +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 -----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 +10)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1616,20 +1616,20 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalescePartitionsExec ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalescePartitionsExec +05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1642,23 +1642,23 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name ---Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) -----TableScan: join_t1 projection=[t1_id, t1_name] -----TableScan: join_t2 projection=[t2_id] +01)Projection: join_t1.t1_id, join_t2.t2_id, join_t1.t1_name +02)--Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +03)----TableScan: join_t1 projection=[t1_id, t1_name] +04)----TableScan: join_t2 projection=[t2_id] physical_plan -ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 -----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 +06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)------------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1673,16 +1673,16 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -----MemoryExec: partitions=1, partition_sizes=[1] -----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1695,21 +1695,21 @@ INNER JOIN join_t2 ON join_t1.t1_id = join_t2.t2_id - cast(11 as INT UNSIGNED) ---- logical_plan -Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)Inner Join: join_t1.t1_id = join_t2.t2_id - UInt32(11) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 ---------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 +09)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -1728,9 +1728,9 @@ from join_t1 inner join join_t2 on join_t1.t1_id + 11 = join_t2.t2_id ---- logical_plan -Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) ---TableScan: join_t1 projection=[t1_id, t1_name] ---TableScan: join_t2 projection=[t2_id] +01)Inner Join: CAST(join_t1.t1_id AS Int64) + Int64(11) = CAST(join_t2.t2_id AS Int64) +02)--TableScan: join_t1 projection=[t1_id, t1_name] +03)--TableScan: join_t2 projection=[t2_id] # Join only with filter @@ -1741,9 +1741,9 @@ from join_t1 inner join join_t2 on join_t1.t1_id * 4 < join_t2.t2_id ---- logical_plan -Inner Join: Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) ---TableScan: join_t1 projection=[t1_id, t1_name] ---TableScan: join_t2 projection=[t2_id] +01)Inner Join: Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) +02)--TableScan: join_t1 projection=[t1_id, t1_name] +03)--TableScan: join_t2 projection=[t2_id] # Type coercion join with filter and equi expr @@ -1755,9 +1755,9 @@ inner join join_t2 on join_t1.t1_id * 5 = join_t2.t2_id and join_t1.t1_id * 4 < join_t2.t2_id ---- logical_plan -Inner Join: CAST(join_t1.t1_id AS Int64) * Int64(5) = CAST(join_t2.t2_id AS Int64) Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) ---TableScan: join_t1 projection=[t1_id, t1_name] ---TableScan: join_t2 projection=[t2_id] +01)Inner Join: CAST(join_t1.t1_id AS Int64) * Int64(5) = CAST(join_t2.t2_id AS Int64) Filter: CAST(join_t1.t1_id AS Int64) * Int64(4) < CAST(join_t2.t2_id AS Int64) +02)--TableScan: join_t1 projection=[t1_id, t1_name] +03)--TableScan: join_t2 projection=[t2_id] # Test cross join to groupby with different key ordering @@ -1810,11 +1810,11 @@ from join_t1 where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +05)------TableScan: join_t2 projection=[t2_id] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1839,12 +1839,12 @@ where join_t1.t1_id + 12 in ) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int -------Filter: join_t2.t2_int > UInt32(0) ---------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int +05)------Filter: join_t2.t2_int > UInt32(0) +06)--------TableScan: join_t2 projection=[t2_id, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1875,12 +1875,12 @@ where join_t1.t1_id + 12 in ) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name -------Filter: join_t2.t2_int > UInt32(0) ---------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name +05)------Filter: join_t2.t2_int > UInt32(0) +06)--------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1907,12 +1907,12 @@ where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2 where join_t1.t1_int > 0) ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) ---Filter: join_t1.t1_int > UInt32(0) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) +02)--Filter: join_t1.t1_int > UInt32(0) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--SubqueryAlias: __correlated_sq_1 +05)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +06)------TableScan: join_t2 projection=[t2_id] # Not in subquery to join with correlated outer filter @@ -1924,11 +1924,11 @@ where join_t1.t1_id + 12 not in (select join_t2.t2_id + 1 from join_t2 where join_t1.t1_int > 0) ---- logical_plan -LeftAnti Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int > UInt32(0) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_id] +01)LeftAnti Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int > UInt32(0) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +05)------TableScan: join_t2 projection=[t2_id] # In subquery to join with outer filter @@ -1946,12 +1946,12 @@ where join_t1.t1_id + 12 in and join_t1.t1_id > 0 ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name ---Filter: join_t1.t1_id > UInt32(0) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name -------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] +01)LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) Filter: join_t1.t1_int <= __correlated_sq_1.t2_int AND join_t1.t1_name != __correlated_sq_1.t2_name +02)--Filter: join_t1.t1_id > UInt32(0) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--SubqueryAlias: __correlated_sq_1 +05)----Projection: CAST(join_t2.t2_id AS Int64) + Int64(1), join_t2.t2_int, join_t2.t2_name +06)------TableScan: join_t2 projection=[t2_id, t2_name, t2_int] query ITI rowsort select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -1979,16 +1979,16 @@ where join_t1.t1_id + 12 in (select join_t2.t2_id + 1 from join_t2) and join_t1.t1_id > 0 ---- logical_plan -LeftSemi Join: CAST(join_t1.t1_int AS Int64) = __correlated_sq_2.join_t2.t2_int + Int64(1) ---LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) -----Filter: join_t1.t1_id > UInt32(0) -------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] -----SubqueryAlias: __correlated_sq_1 -------Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) ---------TableScan: join_t2 projection=[t2_id] ---SubqueryAlias: __correlated_sq_2 -----Projection: CAST(join_t2.t2_int AS Int64) + Int64(1) -------TableScan: join_t2 projection=[t2_int] +01)LeftSemi Join: CAST(join_t1.t1_int AS Int64) = __correlated_sq_2.join_t2.t2_int + Int64(1) +02)--LeftSemi Join: CAST(join_t1.t1_id AS Int64) + Int64(12) = __correlated_sq_1.join_t2.t2_id + Int64(1) +03)----Filter: join_t1.t1_id > UInt32(0) +04)------TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +05)----SubqueryAlias: __correlated_sq_1 +06)------Projection: CAST(join_t2.t2_id AS Int64) + Int64(1) +07)--------TableScan: join_t2 projection=[t2_id] +08)--SubqueryAlias: __correlated_sq_2 +09)----Projection: CAST(join_t2.t2_int AS Int64) + Int64(1) +10)------TableScan: join_t2 projection=[t2_int] query ITI select join_t1.t1_id, join_t1.t1_name, join_t1.t1_int @@ -2020,24 +2020,24 @@ INNER JOIN join_t2 ON join_t1.t1_id > join_t2.t2_id WHERE join_t1.t1_id > 10 AND join_t2.t2_int > 1 ---- logical_plan -Inner Join: Filter: join_t1.t1_id > join_t2.t2_id ---Filter: join_t1.t1_id > UInt32(10) -----TableScan: join_t1 projection=[t1_id] ---Projection: join_t2.t2_id -----Filter: join_t2.t2_int > UInt32(1) -------TableScan: join_t2 projection=[t2_id, t2_int] +01)Inner Join: Filter: join_t1.t1_id > join_t2.t2_id +02)--Filter: join_t1.t1_id > UInt32(10) +03)----TableScan: join_t1 projection=[t1_id] +04)--Projection: join_t2.t2_id +05)----Filter: join_t2.t2_int > UInt32(1) +06)------TableScan: join_t2 projection=[t2_id, t2_int] physical_plan -NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 ---CoalesceBatchesExec: target_batch_size=2 -----FilterExec: t1_id@0 > 10 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] ---CoalescePartitionsExec -----CoalesceBatchesExec: target_batch_size=2 -------ProjectionExec: expr=[t2_id@0 as t2_id] ---------FilterExec: t2_int@1 > 1 -----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 +02)--CoalesceBatchesExec: target_batch_size=2 +03)----FilterExec: t1_id@0 > 10 +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--CoalescePartitionsExec +07)----ProjectionExec: expr=[t2_id@0 as t2_id] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------FilterExec: t2_int@1 > 1 +10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2059,24 +2059,24 @@ RIGHT JOIN (select t2_id from join_t2 where join_t2.t2_id > 11) as join_t2 ON join_t1.t1_id < join_t2.t2_id ---- logical_plan -Right Join: Filter: join_t1.t1_id < join_t2.t2_id ---SubqueryAlias: join_t1 -----Filter: join_t1.t1_id > UInt32(22) -------TableScan: join_t1 projection=[t1_id] ---SubqueryAlias: join_t2 -----Filter: join_t2.t2_id > UInt32(11) -------TableScan: join_t2 projection=[t2_id] +01)Right Join: Filter: join_t1.t1_id < join_t2.t2_id +02)--SubqueryAlias: join_t1 +03)----Filter: join_t1.t1_id > UInt32(22) +04)------TableScan: join_t1 projection=[t1_id] +05)--SubqueryAlias: join_t2 +06)----Filter: join_t2.t2_id > UInt32(11) +07)------TableScan: join_t2 projection=[t2_id] physical_plan -NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 ---CoalescePartitionsExec -----CoalesceBatchesExec: target_batch_size=2 -------FilterExec: t1_id@0 > 22 ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] ---CoalesceBatchesExec: target_batch_size=2 -----FilterExec: t2_id@0 > 11 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------MemoryExec: partitions=1, partition_sizes=[1] +01)NestedLoopJoinExec: join_type=Right, filter=t1_id@0 < t2_id@1 +02)--CoalescePartitionsExec +03)----CoalesceBatchesExec: target_batch_size=2 +04)------FilterExec: t1_id@0 > 22 +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)--CoalesceBatchesExec: target_batch_size=2 +08)----FilterExec: t2_id@0 > 11 +09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id @@ -2113,10 +2113,10 @@ WHERE EXISTS ( ) ---- logical_plan -LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2163,12 +2163,12 @@ WHERE EXISTS ( ) ---- logical_plan -LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Filter: join_t2.t2_int < UInt32(3) ---------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Filter: join_t2.t2_int < UInt32(3) +06)--------TableScan: join_t2 projection=[t2_id, t2_int] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2211,11 +2211,11 @@ WHERE EXISTS ( ) ---- logical_plan -LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---Filter: join_t1.t1_int < UInt32(3) -----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----TableScan: join_t2 projection=[t2_id] +01)LeftSemi Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--Filter: join_t1.t1_int < UInt32(3) +03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +04)--SubqueryAlias: __correlated_sq_1 +05)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2258,10 +2258,10 @@ WHERE NOT EXISTS ( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----TableScan: join_t2 projection=[t2_id] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----TableScan: join_t2 projection=[t2_id] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2304,13 +2304,13 @@ WHERE NOT EXISTS ( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2353,13 +2353,13 @@ WHERE NOT EXISTS( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: join_t2.t2_id + join_t2.t2_int, join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] statement ok set datafusion.optimizer.repartition_joins = false; @@ -2404,13 +2404,13 @@ WHERE NOT EXISTS( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] query ITI SELECT * FROM join_t1 @@ -2435,13 +2435,13 @@ WHERE NOT EXISTS( ) ---- logical_plan -LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) ---TableScan: join_t1 projection=[t1_id, t1_name, t1_int] ---SubqueryAlias: __correlated_sq_1 -----Projection: join_t2.t2_id -------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] ---------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id -----------TableScan: join_t2 projection=[t2_id, t2_int] +01)LeftAnti Join: Filter: CAST(join_t1.t1_id AS Int64) + Int64(1) > CAST(__correlated_sq_1.t2_id AS Int64) * Int64(2) +02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] +03)--SubqueryAlias: __correlated_sq_1 +04)----Projection: join_t2.t2_id +05)------Aggregate: groupBy=[[Int64(1), join_t2.t2_int, join_t2.t2_id]], aggr=[[]] +06)--------Projection: Int64(1), join_t2.t2_int, join_t2.t2_id +07)----------TableScan: join_t2 projection=[t2_id, t2_int] query ITI SELECT * FROM join_t1 @@ -2539,22 +2539,22 @@ query TT EXPLAIN SELECT * FROM test_timestamps_tz_table as t1 JOIN test_timestamps_tz_table as t2 ON t1.millis = t2.millis ---- logical_plan -Inner Join: t1.millis = t2.millis ---SubqueryAlias: t1 -----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] ---SubqueryAlias: t2 -----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] +01)Inner Join: t1.millis = t2.millis +02)--SubqueryAlias: t1 +03)----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] +04)--SubqueryAlias: t2 +05)----TableScan: test_timestamps_tz_table projection=[nanos, micros, millis, secs, names] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 +09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] # left_join_using_2 query II @@ -2613,11 +2613,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 join hashjoin_datatype_table_t2 t2 on t1.c1 = t2.c1 ---- logical_plan -Inner Join: t1.c1 = t2.c1 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Inner Join: t1.c1 = t2.c1 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] # hash_join_with_date32 query DDR?DDR? rowsort @@ -2632,11 +2632,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 left join hashjoin_datatype_table_t2 t2 on t1.c2 = t2.c2 ---- logical_plan -Left Join: t1.c2 = t2.c2 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Left Join: t1.c2 = t2.c2 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] # hash_join_with_date64 query DDR?DDR? rowsort @@ -2653,11 +2653,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 right join hashjoin_datatype_table_t1 t2 on t1.c3 = t2.c3 ---- logical_plan -Right Join: t1.c3 = t2.c3 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +01)Right Join: t1.c3 = t2.c3 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] # hash_join_with_decimal query DDR?DDR? rowsort @@ -2673,11 +2673,11 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 join hashjoin_datatype_table_t1 t2 on t1.c4 = t2.c4 ---- logical_plan -Inner Join: t1.c4 = t2.c4 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +01)Inner Join: t1.c4 = t2.c4 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] # hash_join_with_dictionary query DDR?DDR? rowsort @@ -2712,23 +2712,23 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 join hashjoin_datatype_table_t2 t2 on t1.c1 = t2.c1 ---- logical_plan -Inner Join: t1.c1 = t2.c1 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Inner Join: t1.c1 = t2.c1 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] physical_plan -SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] ---SortExec: expr=[c1@0 ASC] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] ---SortExec: expr=[c1@0 ASC] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortMergeJoin: join_type=Inner, on=[(c1@0, c1@0)] +02)--SortExec: expr=[c1@0 ASC] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)--SortExec: expr=[c1@0 ASC] +08)----CoalesceBatchesExec: target_batch_size=2 +09)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) query DDR?DDR? rowsort @@ -2742,25 +2742,25 @@ query TT explain select * from hashjoin_datatype_table_t1 t1 right join hashjoin_datatype_table_t2 t2 on t1.c3 = t2.c3 ---- logical_plan -Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3 ---SubqueryAlias: t1 -----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] ---SubqueryAlias: t2 -----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] +01)Right Join: CAST(t1.c3 AS Decimal128(10, 2)) = t2.c3 +02)--SubqueryAlias: t1 +03)----TableScan: hashjoin_datatype_table_t1 projection=[c1, c2, c3, c4] +04)--SubqueryAlias: t2 +05)----TableScan: hashjoin_datatype_table_t2 projection=[c1, c2, c3, c4] physical_plan -ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4] ---SortMergeJoin: join_type=Right, on=[(CAST(t1.c3 AS Decimal128(10, 2))@4, c3@2)] -----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 -----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] -----SortExec: expr=[c3@2 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4] +02)--SortMergeJoin: join_type=Right, on=[(CAST(t1.c3 AS Decimal128(10, 2))@4, c3@2)] +03)----SortExec: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 +06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)----SortExec: expr=[c3@2 ASC] +10)------CoalesceBatchesExec: target_batch_size=2 +11)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 +12)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) query DDR?DDR? rowsort @@ -2811,18 +2811,18 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2852,18 +2852,18 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -2914,13 +2914,13 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 WHERE t1_id IN (SELECT t2_id FROM left_semi_anti_join_table_t2 t2) ORDER BY t1_id @@ -2950,13 +2950,13 @@ query TT explain SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query IT SELECT t1_id, t1_name FROM left_semi_anti_join_table_t1 t1 LEFT SEMI JOIN left_semi_anti_join_table_t2 t2 ON (t1_id = t2_id) ORDER BY t1_id @@ -3008,18 +3008,18 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3030,18 +3030,18 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3090,13 +3090,13 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t1 t1 WHERE EXISTS (SELECT * FROM right_semi_anti_join_table_t2 t2 where t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3107,13 +3107,13 @@ query TT explain SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id ---- physical_plan -SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] ---SortExec: expr=[t1_id@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 ---------MemoryExec: partitions=1, partition_sizes=[1] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST] +02)--SortExec: expr=[t1_id@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 +05)--------MemoryExec: partitions=1, partition_sizes=[1] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort SELECT t1_id, t1_name, t1_int FROM right_semi_anti_join_table_t2 t2 RIGHT SEMI JOIN right_semi_anti_join_table_t1 t1 on (t2.t2_id = t1.t1_id and t2.t2_name <> t1.t1_name) ORDER BY t1_id @@ -3176,29 +3176,29 @@ EXPLAIN SELECT * ORDER BY l_table.rn1 ---- logical_plan -Sort: l_table.rn1 ASC NULLS LAST ---Inner Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: l_table.rn1 ASC NULLS LAST +02)--Inner Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +06)----------TableScan: annotated_data projection=[a0, a, b, c, d] +07)----SubqueryAlias: r_table +08)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] ---SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [rn1@5 ASC NULLS LAST] +02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +03)----SortExec: expr=[rn1@5 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +08)--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +10)----SortExec: expr=[a@1 ASC] +11)------CoalesceBatchesExec: target_batch_size=2 +12)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +13)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # sort merge join should propagate ordering equivalence of the right side # for right join. Hence final requirement rn1 ASC is already satisfied at @@ -3212,29 +3212,29 @@ EXPLAIN SELECT * ORDER BY r_table.rn1 ---- logical_plan -Sort: r_table.rn1 ASC NULLS LAST ---Right Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: r_table.rn1 ASC NULLS LAST +02)--Right Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------TableScan: annotated_data projection=[a0, a, b, c, d] +05)----SubqueryAlias: r_table +06)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +07)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] ---SortMergeJoin: join_type=Right, on=[(a@1, a@1)] -----SortExec: expr=[a@1 ASC] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----SortExec: expr=[rn1@5 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] ---------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [rn1@10 ASC NULLS LAST] +02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] +03)----SortExec: expr=[a@1 ASC] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +08)----SortExec: expr=[rn1@5 ASC NULLS LAST] +09)------CoalesceBatchesExec: target_batch_size=2 +10)--------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +13)--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +14)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # SortMergeJoin should add ordering equivalences of # right table as lexicographical append to the global ordering @@ -3250,34 +3250,34 @@ EXPLAIN SELECT * ORDER BY l_table.a ASC NULLS FIRST, l_table.b, l_table.c, r_table.rn1 ---- logical_plan -Sort: l_table.a ASC NULLS FIRST, l_table.b ASC NULLS LAST, l_table.c ASC NULLS LAST, r_table.rn1 ASC NULLS LAST ---Inner Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: l_table.a ASC NULLS FIRST, l_table.b ASC NULLS LAST, l_table.c ASC NULLS LAST, r_table.rn1 ASC NULLS LAST +02)--Inner Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +06)----------TableScan: annotated_data projection=[a0, a, b, c, d] +07)----SubqueryAlias: r_table +08)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +09)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +10)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] ---SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] -----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] -------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -------SortExec: expr=[a@1 ASC] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] +02)--SortExec: expr=[a@1 ASC,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,rn1@11 ASC NULLS LAST] +03)----SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] +04)------SortExec: expr=[a@1 ASC] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +09)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +11)------SortExec: expr=[a@1 ASC] +12)--------CoalesceBatchesExec: target_batch_size=2 +13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 +14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +15)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +16)----------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +17)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true statement ok set datafusion.optimizer.prefer_hash_join = true; @@ -3298,21 +3298,21 @@ EXPLAIN SELECT * ORDER BY r_table.rn1 ---- logical_plan -Sort: r_table.rn1 ASC NULLS LAST ---Inner Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: r_table.rn1 ASC NULLS LAST +02)--Inner Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------TableScan: annotated_data projection=[a0, a, b, c, d] +05)----SubqueryAlias: r_table +06)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +07)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +05)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # hash join should propagate ordering equivalence of the right side for RIGHT ANTI join. # Hence final requirement rn1 ASC is already satisfied at the end of HashJoinExec. @@ -3325,21 +3325,21 @@ EXPLAIN SELECT * ORDER BY r_table.rn1 ---- logical_plan -Sort: r_table.rn1 ASC NULLS LAST ---RightAnti Join: l_table.a = r_table.a -----SubqueryAlias: l_table -------TableScan: annotated_data projection=[a] -----SubqueryAlias: r_table -------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Sort: r_table.rn1 ASC NULLS LAST +02)--RightAnti Join: l_table.a = r_table.a +03)----SubqueryAlias: l_table +04)------TableScan: annotated_data projection=[a] +05)----SubqueryAlias: r_table +06)------Projection: annotated_data.a0, annotated_data.a, annotated_data.b, annotated_data.c, annotated_data.d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +07)--------WindowAggr: windowExpr=[[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +08)----------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], has_header=true -----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=RightAnti, on=[(a@0, a@1)] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a], output_ordering=[a@0 ASC], has_header=true +04)----ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] +05)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 @@ -3350,21 +3350,21 @@ GROUP BY l.a, l.b, l.c ORDER BY l.a ASC NULLS FIRST; ---- logical_plan -Sort: l.a ASC NULLS FIRST ---Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 -----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] -------Inner Join: l.a = r.a ---------SubqueryAlias: l -----------TableScan: annotated_data projection=[a, b, c] ---------SubqueryAlias: r -----------TableScan: annotated_data projection=[a, b] +01)Sort: l.a ASC NULLS FIRST +02)--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +03)----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +04)------Inner Join: l.a = r.a +05)--------SubqueryAlias: l +06)----------TableScan: annotated_data projection=[a, b, c] +07)--------SubqueryAlias: r +08)----------TableScan: annotated_data projection=[a, b] physical_plan -ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true # create a table where there more than one valid ordering # that describes table. @@ -3394,27 +3394,27 @@ GROUP BY row_n ORDER BY row_n ---- logical_plan -Projection: amount_usd ---Sort: row_n ASC NULLS LAST -----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n -------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] ---------Projection: l.a, l.d, row_n -----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) -------------SubqueryAlias: l ---------------TableScan: multiple_ordered_table projection=[a, d] -------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n ---------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------------SubqueryAlias: r -------------------TableScan: multiple_ordered_table projection=[a, d] +01)Projection: amount_usd +02)--Sort: row_n ASC NULLS LAST +03)----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +04)------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +05)--------Projection: l.a, l.d, row_n +06)----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +07)------------SubqueryAlias: l +08)--------------TableScan: multiple_ordered_table projection=[a, d] +09)------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +10)--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +11)----------------SubqueryAlias: r +12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan -ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] ---AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true ---------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # run query above in multiple partitions statement ok @@ -3433,32 +3433,32 @@ GROUP BY l.a, l.b, l.c ORDER BY l.a ASC NULLS FIRST; ---- logical_plan -Sort: l.a ASC NULLS FIRST ---Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 -----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] -------Inner Join: l.a = r.a ---------SubqueryAlias: l -----------TableScan: annotated_data projection=[a, b, c] ---------SubqueryAlias: r -----------TableScan: annotated_data projection=[a, b] +01)Sort: l.a ASC NULLS FIRST +02)--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +03)----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +04)------Inner Join: l.a = r.a +05)--------SubqueryAlias: l +06)----------TableScan: annotated_data projection=[a, b, c] +07)--------SubqueryAlias: r +08)----------TableScan: annotated_data projection=[a, b] physical_plan -SortPreservingMergeExec: [a@0 ASC] ---SortExec: expr=[a@0 ASC] -----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] ---------------CoalesceBatchesExec: target_batch_size=2 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true -------------------CoalesceBatchesExec: target_batch_size=2 ---------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true +01)SortPreservingMergeExec: [a@0 ASC] +02)--SortExec: expr=[a@0 ASC] +03)----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 +07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +10)------------------CoalesceBatchesExec: target_batch_size=2 +11)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +14)------------------CoalesceBatchesExec: target_batch_size=2 +15)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true query TT EXPLAIN SELECT * @@ -3466,16 +3466,16 @@ FROM annotated_data as l, annotated_data as r WHERE l.a > r.a ---- logical_plan -Inner Join: Filter: r.a < l.a ---SubqueryAlias: l -----TableScan: annotated_data projection=[a0, a, b, c, d] ---SubqueryAlias: r -----TableScan: annotated_data projection=[a0, a, b, c, d] +01)Inner Join: Filter: r.a < l.a +02)--SubqueryAlias: l +03)----TableScan: annotated_data projection=[a0, a, b, c, d] +04)--SubqueryAlias: r +05)----TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 ---RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true ---CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 +02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +04)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # Currently datafusion cannot pushdown filter conditions with scalar UDF into # cross join. @@ -3485,19 +3485,19 @@ FROM annotated_data as t1, annotated_data as t2 WHERE EXAMPLE(t1.a, t2.a) > 3 ---- logical_plan -Filter: example(CAST(t1.a AS Float64), CAST(t2.a AS Float64)) > Float64(3) ---CrossJoin: -----SubqueryAlias: t1 -------TableScan: annotated_data projection=[a0, a, b, c, d] -----SubqueryAlias: t2 -------TableScan: annotated_data projection=[a0, a, b, c, d] +01)Filter: example(CAST(t1.a AS Float64), CAST(t2.a AS Float64)) > Float64(3) +02)--CrossJoin: +03)----SubqueryAlias: t1 +04)------TableScan: annotated_data projection=[a0, a, b, c, d] +05)----SubqueryAlias: t2 +06)------TableScan: annotated_data projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=2 ---FilterExec: example(CAST(a@1 AS Float64), CAST(a@6 AS Float64)) > 3 -----CrossJoinExec -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +01)CoalesceBatchesExec: target_batch_size=2 +02)--FilterExec: example(CAST(a@1 AS Float64), CAST(a@6 AS Float64)) > 3 +03)----CrossJoinExec +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true #### # Config teardown From 906b73a8b02bcfb00f134102105494c6a8f35def Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 17:12:16 +0300 Subject: [PATCH 51/85] fixing count mismatch --- .../aggregate_statistics.rs | 7 ++--- .../sqllogictest/test_files/subquery.slt | 26 +++++++++++++++++++ 2 files changed, 28 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index df54222270ce..ae6ad41e6c4e 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -35,9 +35,6 @@ use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; #[derive(Default)] pub struct AggregateStatistics {} -/// The name of the column corresponding to [`COUNT_STAR_EXPANSION`] -const COUNT_STAR_NAME: &str = "COUNT(*)"; - impl AggregateStatistics { #[allow(missing_docs)] pub fn new() -> Self { @@ -144,7 +141,7 @@ fn take_optimizable(node: &dyn ExecutionPlan) -> Option> fn take_optimizable_table_count( agg_expr: &dyn AggregateExpr, stats: &Statistics, -) -> Option<(ScalarValue, &'static str)> { +) -> Option<(ScalarValue, String)> { if let (&Precision::Exact(num_rows), Some(casted_expr)) = ( &stats.num_rows, agg_expr.as_any().downcast_ref::(), @@ -158,7 +155,7 @@ fn take_optimizable_table_count( if lit_expr.value() == &COUNT_STAR_EXPANSION { return Some(( ScalarValue::Int64(Some(num_rows as i64)), - COUNT_STAR_NAME, + casted_expr.name().to_owned(), )); } } diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 477e907cab41..faba074f6fd7 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -726,6 +726,32 @@ Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT( ------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] --------TableScan: t2 projection=[] +statement ok +set datafusion.explain.logical_plan_only = false; + +query TT +explain select (select count(*) from t1) as b, (select count(1) from t2) +---- +logical_plan +01)Projection: __scalar_sq_1.COUNT(*) AS b, __scalar_sq_2.COUNT(Int64(1)) AS COUNT(Int64(1)) +02)--Left Join: +03)----SubqueryAlias: __scalar_sq_1 +04)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)--------TableScan: t1 projection=[] +06)----SubqueryAlias: __scalar_sq_2 +07)------Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1))]] +08)--------TableScan: t2 projection=[] +physical_plan +01)ProjectionExec: expr=[COUNT(*)@0 as b, COUNT(Int64(1))@1 as COUNT(Int64(1))] +02)--NestedLoopJoinExec: join_type=Left +03)----ProjectionExec: expr=[4 as COUNT(*)] +04)------PlaceholderRowExec +05)----ProjectionExec: expr=[4 as COUNT(Int64(1))] +06)------PlaceholderRowExec + +statement ok +set datafusion.explain.logical_plan_only = true; + query II select (select count(*) from t1) as b, (select count(1) from t2) ---- From cac1d1b5fb13e3f25fa3cb1ecb59a2db91b14a63 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 17:14:14 +0300 Subject: [PATCH 52/85] Update aggregate_statistics.rs --- datafusion/core/src/physical_optimizer/aggregate_statistics.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index ae6ad41e6c4e..98f8884e4985 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -424,7 +424,7 @@ pub(crate) mod tests { /// What name would this aggregate produce in a plan? fn column_name(&self) -> &'static str { match self { - Self::CountStar => COUNT_STAR_NAME, + Self::CountStar => "COUNT(*)", Self::ColumnA(_) => "COUNT(a)", } } From 2f60c16bb6d26a31e6dd26430e200180bac7a7f4 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 17:57:53 +0300 Subject: [PATCH 53/85] catch different names --- .../src/equivalence/projection.rs | 5 ++- .../sqllogictest/test_files/aggregate.slt | 42 +++++++++++++++++++ .../sqllogictest/test_files/group_by.slt | 4 +- 3 files changed, 48 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 3ac1bb69f265..4227471d7e93 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; use crate::expressions::Column; use crate::PhysicalExpr; @@ -66,6 +66,9 @@ impl ProjectionMapping { // Conceptually, `source_expr` and `expression` should be the same. let idx = col.index(); let matching_input_field = input_schema.field(idx); + if col.name() != matching_input_field.name() { + return Err(DataFusionError::Internal(format!("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))) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 1310e9fad400..0221d53b9078 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3454,3 +3454,45 @@ SELECT LAST_VALUE(column1 ORDER BY column2 DESC) IGNORE NULLS FROM t; statement ok DROP TABLE t; + +# Test Convert FirstLast optimizer rule +statement ok +CREATE EXTERNAL TABLE convert_first_last_table ( +c1 INT NOT NULL, +c2 INT NOT NULL, +c3 INT NOT NULL +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (c1 ASC) +WITH ORDER (c2 DESC) +WITH ORDER (c3 ASC) +LOCATION '../core/tests/data/convert_first_last.csv'; + +# test first to last, the result does not show difference, we need to check the conversion by `explain` +query TT +explain select first_value(c1 order by c3 desc) from convert_first_last_table; +---- +logical_plan +01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] +02)--TableScan: convert_first_last_table projection=[c1, c3] +physical_plan +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true + +# test last to first +query TT +explain select last_value(c1 order by c2 asc) from convert_first_last_table; +---- +logical_plan +01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] +02)--TableScan: convert_first_last_table projection=[c1, c2] +physical_plan +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 5421c3a57dde..1a9a0cbf9914 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2805,7 +2805,7 @@ logical_plan 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -3800,7 +3800,7 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] From b650e002d74f3521379aadce93170c3051b9a4d6 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 15 Apr 2024 18:50:04 +0300 Subject: [PATCH 54/85] Fix after merge --- .../core/src/physical_optimizer/optimize_projections.rs | 4 ++-- datafusion/physical-expr-common/src/aggregate/mod.rs | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index c2d2e22cf379..be4058ce277e 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -79,7 +79,7 @@ use datafusion_physical_plan::aggregates::{ }; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::insert::FileSinkExec; +use datafusion_physical_plan::insert::DataSinkExec; use datafusion_physical_plan::joins::utils::{ ColumnIndex, JoinFilter, JoinOn, JoinOnRef, }; @@ -3504,7 +3504,7 @@ impl ProjectionOptimizer { return Ok(Transformed::no(self)); }; update_mapping(&mut self, all_mappings) - } else if let Some(_file_sink) = plan_any.downcast_ref::() { + } 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 = diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 33044fd9beee..87576648c462 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -137,6 +137,15 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { fn create_sliding_accumulator(&self) -> Result> { not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") } + + /// Rewrites the aggregate expression with the given expressions. + /// The order of the given expressions is taken into account while replacing. + fn with_new_expressions( + self: Arc, + _expressions: Vec>, + ) -> Option> { + None + } } /// Physical aggregate expression of a UDAF. From 155a53fc4c294c8b017a152d7d3d2f8e6f38842b Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 16 Apr 2024 17:38:24 +0300 Subject: [PATCH 55/85] fix after merge --- .../physical_optimizer/convert_first_last.rs | 4 +- datafusion/core/src/physical_planner.rs | 1 - .../sqllogictest/test_files/aggregate.slt | 20 +- datafusion/sqllogictest/test_files/cte.slt | 10 +- .../sqllogictest/test_files/explain.slt | 13 +- .../sqllogictest/test_files/group_by.slt | 12 +- datafusion/sqllogictest/test_files/joins.slt | 4 +- datafusion/sqllogictest/test_files/limit.slt | 6 +- .../sqllogictest/test_files/predicates.slt | 38 +- .../sqllogictest/test_files/subquery.slt | 48 +- .../sqllogictest/test_files/tpch/q1.slt.part | 36 +- .../sqllogictest/test_files/tpch/q10.slt.part | 112 +- .../sqllogictest/test_files/tpch/q11.slt.part | 158 +- .../sqllogictest/test_files/tpch/q12.slt.part | 54 +- .../sqllogictest/test_files/tpch/q13.slt.part | 70 +- .../sqllogictest/test_files/tpch/q14.slt.part | 48 +- .../sqllogictest/test_files/tpch/q15.slt.part | 110 +- .../sqllogictest/test_files/tpch/q16.slt.part | 98 +- .../sqllogictest/test_files/tpch/q17.slt.part | 76 +- .../sqllogictest/test_files/tpch/q18.slt.part | 94 +- .../sqllogictest/test_files/tpch/q19.slt.part | 54 +- .../sqllogictest/test_files/tpch/q2.slt.part | 234 +-- .../sqllogictest/test_files/tpch/q20.slt.part | 140 +- .../sqllogictest/test_files/tpch/q21.slt.part | 172 +- .../sqllogictest/test_files/tpch/q22.slt.part | 90 +- .../sqllogictest/test_files/tpch/q3.slt.part | 96 +- .../sqllogictest/test_files/tpch/q4.slt.part | 66 +- .../sqllogictest/test_files/tpch/q5.slt.part | 152 +- .../sqllogictest/test_files/tpch/q6.slt.part | 26 +- .../sqllogictest/test_files/tpch/q8.slt.part | 210 +-- .../sqllogictest/test_files/tpch/q9.slt.part | 148 +- datafusion/sqllogictest/test_files/union.slt | 60 +- datafusion/sqllogictest/test_files/window.slt | 1640 ++++++++--------- 33 files changed, 2051 insertions(+), 2049 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 4102313d3126..41e1ddda0be5 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -79,7 +79,7 @@ fn get_common_requirement_of_aggregate_input( if let Some(aggr_exec) = plan.as_any().downcast_ref::() { let input = aggr_exec.input(); let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); - let group_by = aggr_exec.group_by(); + let group_by = aggr_exec.group_expr(); let mode = aggr_exec.mode(); let input_eq_properties = input.equivalence_properties(); @@ -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_planner.rs b/datafusion/core/src/physical_planner.rs index 3116f41c2961..49535168f70f 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -85,7 +85,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::{ expr_vec_fmt, DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index ca04bf78da97..c8791516c7cc 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -2724,16 +2724,16 @@ logical_plan 04)------Aggregate: groupBy=[[aggregate_test_100.c2, aggregate_test_100.c3]], aggr=[[]] 05)--------TableScan: aggregate_test_100 projection=[c2, c3] physical_plan -GlobalLimitExec: skip=10, fetch=3 ---AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[c3@1 as c3, c2@0 as c2], aggr=[], lim=[13] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -------------CoalescePartitionsExec ---------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true +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@1 as c3, c2@0 as c2], aggr=[], lim=[13] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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 8b972bc79b03..f99bb45ad933 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -740,11 +740,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/arrow-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 8296ab5d7dee..23dc10c51581 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -43,11 +43,11 @@ 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 -CoalesceBatchesExec: target_batch_size=8192 ---ProjectionExec: expr=[c1@0 as c1] -----FilterExec: c2@1 > 10 -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true +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 # explain_csv_exec_scan_config @@ -248,6 +248,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE 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 coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after PipelineChecker SAME TEXT AS ABOVE @@ -303,6 +304,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE 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 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]:)]] @@ -338,6 +340,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE 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 coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements 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 1a9a0cbf9914..b44afc0974e8 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -3420,9 +3420,9 @@ logical_plan 08)----------SubqueryAlias: r 09)------------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] -02)--SortExec: expr=[sn@0 ASC NULLS LAST] -03)----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] +01)ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] +02)--SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +03)----SortExec: expr=[sn@0 ASC NULLS LAST] 04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 @@ -3568,9 +3568,9 @@ logical_plan 07)------------SubqueryAlias: l 08)--------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] physical_plan -01)SortPreservingMergeExec: [sn@2 ASC NULLS LAST] -02)--SortExec: expr=[sn@2 ASC NULLS LAST] -03)----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] +01)ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] +02)--SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +03)----SortExec: expr=[sn@0 ASC NULLS LAST] 04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index d96e58ddf20c..92a9e0a6dd9b 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2033,8 +2033,8 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------MemoryExec: partitions=1, partition_sizes=[1] 06)--CoalescePartitionsExec -07)----ProjectionExec: expr=[t2_id@0 as t2_id] -08)------CoalesceBatchesExec: target_batch_size=2 +07)----CoalesceBatchesExec: target_batch_size=2 +08)------ProjectionExec: expr=[t2_id@0 as t2_id] 09)--------FilterExec: t2_int@1 > 1 10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index b4138f38ea2b..183c2bdc5251 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 9d2bf8678fce..7f46f6fbd303 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -726,25 +726,25 @@ logical_plan 09)--------------TableScan: part projection=[p_partkey, p_brand], partial_filters=[part.p_brand = Utf8("Brand#12") OR part.p_brand = Utf8("Brand#23")] 10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan -AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] ---CoalesceBatchesExec: target_batch_size=8192 -----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] -------CoalesceBatchesExec: target_batch_size=8192 ---------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------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 +17)------CoalesceBatchesExec: target_batch_size=8192 +18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +19)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index b7d01c21ea76..9297fd7ef6d3 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -187,17 +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 -ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t1_id@2] -------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] ---------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +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@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 @@ -291,19 +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 -ProjectionExec: expr=[t1_id@1 as t1_id, SUM(t2.t2_int)@0 as t2_sum] ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t1_id@2] -------CoalesceBatchesExec: target_batch_size=2 ---------FilterExec: SUM(t2.t2_int)@1 < 3 -----------AggregateExec: mode=FinalPartitioned, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -------------CoalesceBatchesExec: target_batch_size=2 ---------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -----------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[SUM(t2.t2_int)] -------------------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] +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@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 816bc7bf7661..8464fee16e1e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -40,25 +40,25 @@ explain select l_linestatus; ---- logical_plan -Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST ---Projection: lineitem.l_returnflag, lineitem.l_linestatus, SUM(lineitem.l_quantity) AS sum_qty, SUM(lineitem.l_extendedprice) AS sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(*) AS count_order -----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS 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(Int64(1)) AS COUNT(*)]] -------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus ---------Filter: lineitem.l_shipdate <= Date32("10471") -----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("10471")] +01)Sort: lineitem.l_returnflag ASC NULLS LAST, lineitem.l_linestatus ASC NULLS LAST +02)--Projection: lineitem.l_returnflag, lineitem.l_linestatus, SUM(lineitem.l_quantity) AS sum_qty, SUM(lineitem.l_extendedprice) AS sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax) AS sum_charge, AVG(lineitem.l_quantity) AS avg_qty, AVG(lineitem.l_extendedprice) AS avg_price, AVG(lineitem.l_discount) AS avg_disc, COUNT(*) AS count_order +03)----Aggregate: groupBy=[[lineitem.l_returnflag, lineitem.l_linestatus]], aggr=[[SUM(lineitem.l_quantity), SUM(lineitem.l_extendedprice), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice AS lineitem.l_extendedprice * Decimal128(Some(1),20,0) - lineitem.l_discount * (Decimal128(Some(1),20,0) + lineitem.l_tax)) AS 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(Int64(1)) AS COUNT(*)]] +04)------Projection: lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)Decimal128(Some(1),20,0) - lineitem.l_discountlineitem.l_discountDecimal128(Some(1),20,0)lineitem.l_extendedprice, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus +05)--------Filter: lineitem.l_shipdate <= Date32("10471") +06)----------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate], partial_filters=[lineitem.l_shipdate <= Date32("10471")] physical_plan -SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] ---SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] -----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(*)@9 as count_order] -------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 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(*)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 -------------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(*)] ---------------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_discountlineitem.l_discountDecimal128(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] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------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] ---------------------FilterExec: l_shipdate@6 <= 10471 -----------------------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 +01)SortPreservingMergeExec: [l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] +02)--SortExec: expr=[l_returnflag@0 ASC NULLS LAST,l_linestatus@1 ASC NULLS LAST] +03)----ProjectionExec: expr=[l_returnflag@0 as l_returnflag, l_linestatus@1 as l_linestatus, SUM(lineitem.l_quantity)@2 as sum_qty, SUM(lineitem.l_extendedprice)@3 as sum_base_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@4 as sum_disc_price, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount * Int64(1) + lineitem.l_tax)@5 as sum_charge, AVG(lineitem.l_quantity)@6 as avg_qty, AVG(lineitem.l_extendedprice)@7 as avg_price, AVG(lineitem.l_discount)@8 as avg_disc, COUNT(*)@9 as count_order] +04)------AggregateExec: mode=FinalPartitioned, gby=[l_returnflag@0 as l_returnflag, l_linestatus@1 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(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([l_returnflag@0, l_linestatus@1], 4), input_partitions=4 +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_discountlineitem.l_discountDecimal128(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)------------------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 b04f2538a8a0..1515040eccd0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -51,63 +51,63 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: revenue DESC NULLS FIRST, fetch=10 -----Projection: customer.c_custkey, customer.c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment -------Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] ---------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name -----------Inner Join: customer.c_nationkey = nation.n_nationkey -------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount ---------------Inner Join: orders.o_orderkey = lineitem.l_orderkey -----------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey -------------------Inner Join: customer.c_custkey = orders.o_custkey ---------------------TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] ---------------------Projection: orders.o_orderkey, orders.o_custkey -----------------------Filter: orders.o_orderdate >= Date32("8674") AND orders.o_orderdate < Date32("8766") -------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8674"), orders.o_orderdate < Date32("8766")] -----------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount -------------------Filter: lineitem.l_returnflag = Utf8("R") ---------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8("R")] -------------TableScan: nation projection=[n_nationkey, n_name] +01)Limit: skip=0, fetch=10 +02)--Sort: revenue DESC NULLS FIRST, fetch=10 +03)----Projection: customer.c_custkey, customer.c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment +04)------Aggregate: groupBy=[[customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +05)--------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name +06)----------Inner Join: customer.c_nationkey = nation.n_nationkey +07)------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount +08)--------------Inner Join: orders.o_orderkey = lineitem.l_orderkey +09)----------------Projection: customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey +10)------------------Inner Join: customer.c_custkey = orders.o_custkey +11)--------------------TableScan: customer projection=[c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_comment] +12)--------------------Projection: orders.o_orderkey, orders.o_custkey +13)----------------------Filter: orders.o_orderdate >= Date32("8674") AND orders.o_orderdate < Date32("8766") +14)------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8674"), orders.o_orderdate < Date32("8766")] +15)----------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +16)------------------Filter: lineitem.l_returnflag = Utf8("R") +17)--------------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_returnflag], partial_filters=[lineitem.l_returnflag = Utf8("R")] +18)------------TableScan: nation projection=[n_nationkey, n_name] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [revenue@2 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[revenue@2 DESC] -------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] ---------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------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 -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ---------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 -----------------------------------------------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 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -------------------------------------FilterExec: l_returnflag@3 = R ---------------------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [revenue@2 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[revenue@2 DESC] +04)------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@7 as revenue, c_acctbal@2 as c_acctbal, n_name@4 as n_name, c_address@5 as c_address, c_phone@3 as c_phone, c_comment@6 as c_comment] +05)--------AggregateExec: mode=FinalPartitioned, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@2 as c_acctbal, c_phone@3 as c_phone, n_name@4 as n_name, c_address@5 as c_address, c_comment@6 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +21)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)----------------------------------------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)--------------------------------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 +36)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +37)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)--------------------------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/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 837171447e7c..e0b53a9b3440 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -47,86 +47,86 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: value DESC NULLS FIRST, fetch=10 -----Projection: partsupp.ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS value -------Inner Join: Filter: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > __scalar_sq_1.SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001) ---------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] -----------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost -------------Inner Join: supplier.s_nationkey = nation.n_nationkey ---------------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey -----------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey -------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] -------------------TableScan: supplier projection=[s_suppkey, s_nationkey] ---------------Projection: nation.n_nationkey -----------------Filter: nation.n_name = Utf8("GERMANY") -------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) -------------Aggregate: groupBy=[[]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] ---------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost -----------------Inner Join: supplier.s_nationkey = nation.n_nationkey -------------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey ---------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey -----------------------TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] -----------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -------------------Projection: nation.n_nationkey ---------------------Filter: nation.n_name = Utf8("GERMANY") -----------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] +01)Limit: skip=0, fetch=10 +02)--Sort: value DESC NULLS FIRST, fetch=10 +03)----Projection: partsupp.ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS value +04)------Inner Join: Filter: CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Decimal128(38, 15)) > __scalar_sq_1.SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001) +05)--------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] +06)----------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost +07)------------Inner Join: supplier.s_nationkey = nation.n_nationkey +08)--------------Projection: partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey +09)----------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +10)------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty, ps_supplycost] +11)------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +12)--------------Projection: nation.n_nationkey +13)----------------Filter: nation.n_name = Utf8("GERMANY") +14)------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] +15)--------SubqueryAlias: __scalar_sq_1 +16)----------Projection: CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty) AS Float64) * Float64(0.0001) AS Decimal128(38, 15)) +17)------------Aggregate: groupBy=[[]], aggr=[[SUM(partsupp.ps_supplycost * CAST(partsupp.ps_availqty AS Decimal128(10, 0)))]] +18)--------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost +19)----------------Inner Join: supplier.s_nationkey = nation.n_nationkey +20)------------------Projection: partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey +21)--------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +22)----------------------TableScan: partsupp projection=[ps_suppkey, ps_availqty, ps_supplycost] +23)----------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +24)------------------Projection: nation.n_nationkey +25)--------------------Filter: nation.n_name = Utf8("GERMANY") +26)----------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("GERMANY")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [value@1 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[value@1 DESC] -------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] ---------NestedLoopJoinExec: join_type=Inner, filter=CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 -----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -----------------------------------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, ps_supplycost], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------------FilterExec: n_name@1 = GERMANY ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -----------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------------FilterExec: n_name@1 = GERMANY ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [value@1 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[value@1 DESC] +04)------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@1 as value] +05)--------NestedLoopJoinExec: join_type=Inner, filter=CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Decimal128(38, 15)) > SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)@1 +06)----------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +09)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +18)----------------------------------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, ps_supplycost], has_header=false +19)------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +21)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------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 +30)----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +31)------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)--------------CoalescePartitionsExec +33)----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +34)------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +36)----------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +38)--------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +40)------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +42)----------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false +43)------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +45)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------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 8b3477cbd6f7..a32cac928fe2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -49,34 +49,34 @@ order by l_shipmode; ---- logical_plan -Sort: lineitem.l_shipmode ASC NULLS LAST ---Projection: lineitem.l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS low_line_count -----Aggregate: groupBy=[[lineitem.l_shipmode]], aggr=[[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]] -------Projection: lineitem.l_shipmode, orders.o_orderpriority ---------Inner Join: lineitem.l_orderkey = orders.o_orderkey -----------Projection: lineitem.l_orderkey, lineitem.l_shipmode -------------Filter: (lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("8766") AND lineitem.l_receiptdate < Date32("9131") ---------------TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("8766"), lineitem.l_receiptdate < Date32("9131")] -----------TableScan: orders projection=[o_orderkey, o_orderpriority] +01)Sort: lineitem.l_shipmode ASC NULLS LAST +02)--Projection: lineitem.l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END) AS low_line_count +03)----Aggregate: groupBy=[[lineitem.l_shipmode]], aggr=[[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)]] +04)------Projection: lineitem.l_shipmode, orders.o_orderpriority +05)--------Inner Join: lineitem.l_orderkey = orders.o_orderkey +06)----------Projection: lineitem.l_orderkey, lineitem.l_shipmode +07)------------Filter: (lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP")) AND lineitem.l_receiptdate > lineitem.l_commitdate AND lineitem.l_shipdate < lineitem.l_commitdate AND lineitem.l_receiptdate >= Date32("8766") AND lineitem.l_receiptdate < Date32("9131") +08)--------------TableScan: lineitem projection=[l_orderkey, l_shipdate, l_commitdate, l_receiptdate, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("MAIL") OR lineitem.l_shipmode = Utf8("SHIP"), lineitem.l_receiptdate > lineitem.l_commitdate, lineitem.l_shipdate < lineitem.l_commitdate, lineitem.l_receiptdate >= Date32("8766"), lineitem.l_receiptdate < Date32("9131")] +09)----------TableScan: orders projection=[o_orderkey, o_orderpriority] physical_plan -SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] ---SortExec: expr=[l_shipmode@0 ASC NULLS LAST] -----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] -------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] ---------------------------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 -----------------------------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 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------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_orderpriority], has_header=false +01)SortPreservingMergeExec: [l_shipmode@0 ASC NULLS LAST] +02)--SortExec: expr=[l_shipmode@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[l_shipmode@0 as l_shipmode, SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@1 as high_line_count, SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)@2 as low_line_count] +04)------AggregateExec: mode=FinalPartitioned, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +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 +17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +18)----------------------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_orderpriority], has_header=false query TII diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index 2779b5996260..e5ea885d9162 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -40,42 +40,42 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST, fetch=10 -----Projection: c_orders.c_count, COUNT(*) AS custdist -------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] ---------SubqueryAlias: c_orders -----------Projection: COUNT(orders.o_orderkey) AS c_count -------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] ---------------Projection: customer.c_custkey, orders.o_orderkey -----------------Left Join: customer.c_custkey = orders.o_custkey -------------------TableScan: customer projection=[c_custkey] -------------------Projection: orders.o_orderkey, orders.o_custkey ---------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") -----------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] +01)Limit: skip=0, fetch=10 +02)--Sort: custdist DESC NULLS FIRST, c_orders.c_count DESC NULLS FIRST, fetch=10 +03)----Projection: c_orders.c_count, COUNT(*) AS custdist +04)------Aggregate: groupBy=[[c_orders.c_count]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +05)--------SubqueryAlias: c_orders +06)----------Projection: COUNT(orders.o_orderkey) AS c_count +07)------------Aggregate: groupBy=[[customer.c_custkey]], aggr=[[COUNT(orders.o_orderkey)]] +08)--------------Projection: customer.c_custkey, orders.o_orderkey +09)----------------Left Join: customer.c_custkey = orders.o_custkey +10)------------------TableScan: customer projection=[c_custkey] +11)------------------Projection: orders.o_orderkey, orders.o_custkey +12)--------------------Filter: orders.o_comment NOT LIKE Utf8("%special%requests%") +13)----------------------TableScan: orders projection=[o_orderkey, o_custkey, o_comment], partial_filters=[orders.o_comment NOT LIKE Utf8("%special%requests%")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC] -------ProjectionExec: expr=[c_count@0 as c_count, COUNT(*)@1 as custdist] ---------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(*)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] -----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] -------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] ---------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -----------------------------------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 +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [custdist@1 DESC,c_count@0 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[custdist@1 DESC,c_count@0 DESC] +04)------ProjectionExec: expr=[c_count@0 as c_count, COUNT(*)@1 as custdist] +05)--------AggregateExec: mode=FinalPartitioned, gby=[c_count@0 as c_count], aggr=[COUNT(*)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([c_count@0], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] +09)----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] +10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)----------------------------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 ee48940caa95..d53fe0b3177e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -32,31 +32,31 @@ where and l_shipdate < date '1995-10-01'; ---- logical_plan -Projection: Float64(100) * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END) AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS Float64) AS promo_revenue ---Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_type -------Inner Join: lineitem.l_partkey = part.p_partkey ---------Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount -----------Filter: lineitem.l_shipdate >= Date32("9374") AND lineitem.l_shipdate < Date32("9404") -------------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9374"), lineitem.l_shipdate < Date32("9404")] ---------TableScan: part projection=[p_partkey, p_type] +01)Projection: Float64(100) * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END) AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS Float64) AS promo_revenue +02)--Aggregate: groupBy=[[]], aggr=[[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +03)----Projection: lineitem.l_extendedprice, lineitem.l_discount, part.p_type +04)------Inner Join: lineitem.l_partkey = part.p_partkey +05)--------Projection: lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount +06)----------Filter: lineitem.l_shipdate >= Date32("9374") AND lineitem.l_shipdate < Date32("9404") +07)------------TableScan: lineitem projection=[l_partkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9374"), lineitem.l_shipdate < Date32("9404")] +08)--------TableScan: part projection=[p_partkey, p_type] physical_plan -ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_type@4] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 -----------------------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 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +01)ProjectionExec: expr=[100 * CAST(SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END)@0 AS Float64) / CAST(SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 AS Float64) as promo_revenue] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +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)----------------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 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 71f869dd0b33..51d45ba5ff4c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -50,62 +50,62 @@ order by s_suppkey; ---- logical_plan -Sort: supplier.s_suppkey ASC NULLS LAST ---Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue -----Inner Join: revenue0.total_revenue = __scalar_sq_1.MAX(revenue0.total_revenue) -------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue ---------Inner Join: supplier.s_suppkey = revenue0.supplier_no -----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] -----------SubqueryAlias: revenue0 -------------Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue ---------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") ---------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] -------SubqueryAlias: __scalar_sq_1 ---------Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] -----------SubqueryAlias: revenue0 -------------Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue ---------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") ---------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] +01)Sort: supplier.s_suppkey ASC NULLS LAST +02)--Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue +03)----Inner Join: revenue0.total_revenue = __scalar_sq_1.MAX(revenue0.total_revenue) +04)------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue0.total_revenue +05)--------Inner Join: supplier.s_suppkey = revenue0.supplier_no +06)----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_phone] +07)----------SubqueryAlias: revenue0 +08)------------Projection: lineitem.l_suppkey AS supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue +09)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +10)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +11)------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") +12)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] +13)------SubqueryAlias: __scalar_sq_1 +14)--------Aggregate: groupBy=[[]], aggr=[[MAX(revenue0.total_revenue)]] +15)----------SubqueryAlias: revenue0 +16)------------Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS total_revenue +17)--------------Aggregate: groupBy=[[lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +18)----------------Projection: lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +19)------------------Filter: lineitem.l_shipdate >= Date32("9496") AND lineitem.l_shipdate < Date32("9587") +20)--------------------TableScan: lineitem projection=[l_suppkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("9496"), lineitem.l_shipdate < Date32("9587")] physical_plan -SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] ---SortExec: expr=[s_suppkey@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 -------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false -----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 ---------------------------------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 ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 -------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] ---------------CoalescePartitionsExec -----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] -------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] ---------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 ---------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] ---------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -----------------------------------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 +01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] +02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +13)----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +14)------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +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)--------------------------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 +23)----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +24)------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +25)--------------CoalescePartitionsExec +26)----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +27)------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +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)----------------------------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 query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 8a14d77878f9..8c679421ce50 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -50,56 +50,56 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 -----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt -------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] ---------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] -----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey -------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size ---------------Inner Join: partsupp.ps_partkey = part.p_partkey -----------------TableScan: partsupp projection=[ps_partkey, ps_suppkey] -----------------Filter: part.p_brand != Utf8("Brand#45") AND part.p_type NOT LIKE Utf8("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) -------------------TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#45"), part.p_type NOT LIKE Utf8("MEDIUM POLISHED%"), part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)])] -------------SubqueryAlias: __correlated_sq_1 ---------------Projection: supplier.s_suppkey -----------------Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") -------------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] +01)Limit: skip=0, fetch=10 +02)--Sort: supplier_cnt DESC NULLS FIRST, part.p_brand ASC NULLS LAST, part.p_type ASC NULLS LAST, part.p_size ASC NULLS LAST, fetch=10 +03)----Projection: part.p_brand, part.p_type, part.p_size, COUNT(alias1) AS supplier_cnt +04)------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size]], aggr=[[COUNT(alias1)]] +05)--------Aggregate: groupBy=[[part.p_brand, part.p_type, part.p_size, partsupp.ps_suppkey AS alias1]], aggr=[[]] +06)----------LeftAnti Join: partsupp.ps_suppkey = __correlated_sq_1.s_suppkey +07)------------Projection: partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size +08)--------------Inner Join: partsupp.ps_partkey = part.p_partkey +09)----------------TableScan: partsupp projection=[ps_partkey, ps_suppkey] +10)----------------Filter: part.p_brand != Utf8("Brand#45") AND part.p_type NOT LIKE Utf8("MEDIUM POLISHED%") AND part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)]) +11)------------------TableScan: part projection=[p_partkey, p_brand, p_type, p_size], partial_filters=[part.p_brand != Utf8("Brand#45"), part.p_type NOT LIKE Utf8("MEDIUM POLISHED%"), part.p_size IN ([Int32(49), Int32(14), Int32(23), Int32(45), Int32(19), Int32(3), Int32(36), Int32(9)])] +12)------------SubqueryAlias: __correlated_sq_1 +13)--------------Projection: supplier.s_suppkey +14)----------------Filter: supplier.s_comment LIKE Utf8("%Customer%Complaints%") +15)------------------TableScan: supplier projection=[s_suppkey, s_comment], partial_filters=[supplier.s_comment LIKE Utf8("%Customer%Complaints%")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 -----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] -------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] ---------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] -----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------------------------------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], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) ---------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------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 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[supplier_cnt@3 DESC,p_brand@0 ASC NULLS LAST,p_type@1 ASC NULLS LAST,p_size@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, COUNT(alias1)@3 as supplier_cnt] +05)--------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size], aggr=[COUNT(alias1)] +09)----------------AggregateExec: mode=FinalPartitioned, gby=[p_brand@0 as p_brand, p_type@1 as p_type, p_size@2 as p_size, alias1@3 as alias1], aggr=[] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([p_brand@0, p_type@1, p_size@2, alias1@3], 4), input_partitions=4 +12)----------------------AggregateExec: mode=Partial, gby=[p_brand@1 as p_brand, p_type@2 as p_type, p_size@3 as p_size, ps_suppkey@0 as alias1], aggr=[] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +21)----------------------------------------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], has_header=false +22)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +24)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) +26)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------------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 bb4ed35af486..4ca2eee20910 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -36,45 +36,45 @@ where ); ---- logical_plan -Projection: CAST(SUM(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly ---Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice)]] -----Projection: lineitem.l_extendedprice -------Inner Join: part.p_partkey = __scalar_sq_1.l_partkey Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < __scalar_sq_1.Float64(0.2) * AVG(lineitem.l_quantity) ---------Projection: lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey -----------Inner Join: lineitem.l_partkey = part.p_partkey -------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] -------------Projection: part.p_partkey ---------------Filter: part.p_brand = Utf8("Brand#23") AND part.p_container = Utf8("MED BOX") -----------------TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8("Brand#23"), part.p_container = Utf8("MED BOX")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: CAST(Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey -------------Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] ---------------TableScan: lineitem projection=[l_partkey, l_quantity] +01)Projection: CAST(SUM(lineitem.l_extendedprice) AS Float64) / Float64(7) AS avg_yearly +02)--Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice)]] +03)----Projection: lineitem.l_extendedprice +04)------Inner Join: part.p_partkey = __scalar_sq_1.l_partkey Filter: CAST(lineitem.l_quantity AS Decimal128(30, 15)) < __scalar_sq_1.Float64(0.2) * AVG(lineitem.l_quantity) +05)--------Projection: lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey +06)----------Inner Join: lineitem.l_partkey = part.p_partkey +07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice] +08)------------Projection: part.p_partkey +09)--------------Filter: part.p_brand = Utf8("Brand#23") AND part.p_container = Utf8("MED BOX") +10)----------------TableScan: part projection=[p_partkey, p_brand, p_container], partial_filters=[part.p_brand = Utf8("Brand#23"), part.p_container = Utf8("MED BOX")] +11)--------SubqueryAlias: __scalar_sq_1 +12)----------Projection: CAST(Float64(0.2) * CAST(AVG(lineitem.l_quantity) AS Float64) AS Decimal128(30, 15)), lineitem.l_partkey +13)------------Aggregate: groupBy=[[lineitem.l_partkey]], aggr=[[AVG(lineitem.l_quantity)]] +14)--------------TableScan: lineitem projection=[l_partkey, l_quantity] physical_plan -ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------------------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 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------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 -------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] ---------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -----------------------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], has_header=false +01)ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as avg_yearly] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +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)--------------------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 +19)------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] +20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +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_partkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index ed639b71bc3d..7917a1b065e3 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -51,54 +51,54 @@ order by o_orderdate; ---- logical_plan -Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST ---Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] -----LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey -------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity ---------Inner Join: orders.o_orderkey = lineitem.l_orderkey -----------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate -------------Inner Join: customer.c_custkey = orders.o_custkey ---------------TableScan: customer projection=[c_custkey, c_name] ---------------TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] -----------TableScan: lineitem projection=[l_orderkey, l_quantity] -------SubqueryAlias: __correlated_sq_1 ---------Projection: lineitem.l_orderkey -----------Filter: SUM(lineitem.l_quantity) > Decimal128(Some(30000),25,2) -------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] ---------------TableScan: lineitem projection=[l_orderkey, l_quantity] +01)Sort: orders.o_totalprice DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST +02)--Aggregate: groupBy=[[customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice]], aggr=[[SUM(lineitem.l_quantity)]] +03)----LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey +04)------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity +05)--------Inner Join: orders.o_orderkey = lineitem.l_orderkey +06)----------Projection: customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate +07)------------Inner Join: customer.c_custkey = orders.o_custkey +08)--------------TableScan: customer projection=[c_custkey, c_name] +09)--------------TableScan: orders projection=[o_orderkey, o_custkey, o_totalprice, o_orderdate] +10)----------TableScan: lineitem projection=[l_orderkey, l_quantity] +11)------SubqueryAlias: __correlated_sq_1 +12)--------Projection: lineitem.l_orderkey +13)----------Filter: SUM(lineitem.l_quantity) > Decimal128(Some(30000),25,2) +14)------------Aggregate: groupBy=[[lineitem.l_orderkey]], aggr=[[SUM(lineitem.l_quantity)]] +15)--------------TableScan: lineitem projection=[l_orderkey, l_quantity] physical_plan -SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] ---SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] -----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] -------CoalesceBatchesExec: target_batch_size=8192 ---------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 -----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ---------------------------------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_totalprice, o_orderdate], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------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 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] ---------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 -----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -------------------------------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 +01)SortPreservingMergeExec: [o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] +02)--SortExec: expr=[o_totalprice@4 DESC,o_orderdate@3 ASC NULLS LAST] +03)----AggregateExec: mode=FinalPartitioned, gby=[c_name@0 as c_name, c_custkey@1 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@3 as o_orderdate, o_totalprice@4 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([c_name@0, c_custkey@1, o_orderkey@2, o_orderdate@3, o_totalprice@4], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +21)--------------------------------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_totalprice, o_orderdate], has_header=false +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)----------------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 +30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] +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_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 6bd729221d04..51150d6f2f50 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -54,34 +54,34 @@ where ); ---- logical_plan -Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue ---Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -----Projection: lineitem.l_extendedprice, lineitem.l_discount -------Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) ---------Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount -----------Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") -------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG"), lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] ---------Filter: (part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) -----------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)] +01)Projection: SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue +02)--Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +03)----Projection: lineitem.l_extendedprice, lineitem.l_discount +04)------Inner Join: lineitem.l_partkey = part.p_partkey Filter: part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2) AND part.p_size <= Int32(15) +05)--------Projection: lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount +06)----------Filter: (lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)) AND (lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG")) AND lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON") +07)------------TableScan: lineitem projection=[l_partkey, l_quantity, l_extendedprice, l_discount, l_shipinstruct, l_shipmode], partial_filters=[lineitem.l_shipmode = Utf8("AIR") OR lineitem.l_shipmode = Utf8("AIR REG"), lineitem.l_shipinstruct = Utf8("DELIVER IN PERSON"), lineitem.l_quantity >= Decimal128(Some(100),15,2) AND lineitem.l_quantity <= Decimal128(Some(1100),15,2) OR lineitem.l_quantity >= Decimal128(Some(1000),15,2) AND lineitem.l_quantity <= Decimal128(Some(2000),15,2) OR lineitem.l_quantity >= Decimal128(Some(2000),15,2) AND lineitem.l_quantity <= Decimal128(Some(3000),15,2)] +08)--------Filter: (part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) +09)----------TableScan: part projection=[p_partkey, p_brand, p_size, p_container], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_container IN ([Utf8("SM CASE"), Utf8("SM BOX"), Utf8("SM PACK"), Utf8("SM PKG")]) AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_container IN ([Utf8("MED BAG"), Utf8("MED BOX"), Utf8("MED PKG"), Utf8("MED PACK")]) AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_container IN ([Utf8("LG CASE"), Utf8("LG BOX"), Utf8("LG PACK"), Utf8("LG PKG")]) AND part.p_size <= Int32(15)] physical_plan -ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------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] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------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] ---------------------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 -----------------------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 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +01)ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@0 as revenue] +02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND 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)----------------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 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 +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_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index 2dee58a37104..f78ff0b6e8ec 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -63,124 +63,124 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 -----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment -------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) ---------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name -----------Inner Join: nation.n_regionkey = region.r_regionkey -------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey ---------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost -------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey ---------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost -----------------------Inner Join: part.p_partkey = partsupp.ps_partkey -------------------------Projection: part.p_partkey, part.p_mfgr ---------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") -----------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8("%BRASS")] -------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] ---------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] -----------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -------------Projection: region.r_regionkey ---------------Filter: region.r_name = Utf8("EUROPE") -----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] ---------SubqueryAlias: __scalar_sq_1 -----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey -------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] ---------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost -----------------Inner Join: nation.n_regionkey = region.r_regionkey -------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey ---------------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey -------------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey ---------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] ---------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------TableScan: nation projection=[n_nationkey, n_regionkey] -------------------Projection: region.r_regionkey ---------------------Filter: region.r_name = Utf8("EUROPE") -----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] +01)Limit: skip=0, fetch=10 +02)--Sort: supplier.s_acctbal DESC NULLS FIRST, nation.n_name ASC NULLS LAST, supplier.s_name ASC NULLS LAST, part.p_partkey ASC NULLS LAST, fetch=10 +03)----Projection: supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment +04)------Inner Join: part.p_partkey = __scalar_sq_1.ps_partkey, partsupp.ps_supplycost = __scalar_sq_1.MIN(partsupp.ps_supplycost) +05)--------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name +06)----------Inner Join: nation.n_regionkey = region.r_regionkey +07)------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost, nation.n_name, nation.n_regionkey +08)--------------Inner Join: supplier.s_nationkey = nation.n_nationkey +09)----------------Projection: part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_supplycost +10)------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +11)--------------------Projection: part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost +12)----------------------Inner Join: part.p_partkey = partsupp.ps_partkey +13)------------------------Projection: part.p_partkey, part.p_mfgr +14)--------------------------Filter: part.p_size = Int32(15) AND part.p_type LIKE Utf8("%BRASS") +15)----------------------------TableScan: part projection=[p_partkey, p_mfgr, p_type, p_size], partial_filters=[part.p_size = Int32(15), part.p_type LIKE Utf8("%BRASS")] +16)------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +17)--------------------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] +18)----------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +19)------------Projection: region.r_regionkey +20)--------------Filter: region.r_name = Utf8("EUROPE") +21)----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] +22)--------SubqueryAlias: __scalar_sq_1 +23)----------Projection: MIN(partsupp.ps_supplycost), partsupp.ps_partkey +24)------------Aggregate: groupBy=[[partsupp.ps_partkey]], aggr=[[MIN(partsupp.ps_supplycost)]] +25)--------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost +26)----------------Inner Join: nation.n_regionkey = region.r_regionkey +27)------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey +28)--------------------Inner Join: supplier.s_nationkey = nation.n_nationkey +29)----------------------Projection: partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey +30)------------------------Inner Join: partsupp.ps_suppkey = supplier.s_suppkey +31)--------------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +32)--------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +33)----------------------TableScan: nation projection=[n_nationkey, n_regionkey] +34)------------------Projection: region.r_regionkey +35)--------------------Filter: region.r_name = Utf8("EUROPE") +36)----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] physical_plan -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] ---GlobalLimitExec: skip=0, fetch=10 -----SortPreservingMergeExec: [s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST], fetch=10 -------SortExec: TopK(fetch=10), expr=[s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=8192 -----------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] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([p_partkey@0, ps_supplycost@7], 4), input_partitions=4 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------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] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------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] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------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] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -----------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------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 ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -------------------------------------------------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 -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------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 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -----------------------------FilterExec: r_name@1 = EUROPE -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 -----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)], projection=[ps_partkey@0, ps_supplycost@1] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_partkey@0, ps_supplycost@1, n_regionkey@4] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_supplycost@2, s_nationkey@4] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -------------------------------------------------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 ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -------------------------------------FilterExec: r_name@1 = EUROPE ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +01)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] +02)--GlobalLimitExec: skip=0, fetch=10 +03)----SortPreservingMergeExec: [s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST], fetch=10 +04)------SortExec: TopK(fetch=10), expr=[s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST] +05)--------CoalesceBatchesExec: target_batch_size=8192 +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 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] +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@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@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)----------------------------------------------------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 b68828be9ecb..8e081ac4e94f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -56,77 +56,77 @@ order by s_name; ---- logical_plan -Sort: supplier.s_name ASC NULLS LAST ---Projection: supplier.s_name, supplier.s_address -----LeftSemi Join: supplier.s_suppkey = __correlated_sq_1.ps_suppkey -------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address ---------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] -----------Projection: nation.n_nationkey -------------Filter: nation.n_name = Utf8("CANADA") ---------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("CANADA")] -------SubqueryAlias: __correlated_sq_1 ---------Projection: partsupp.ps_suppkey -----------Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * SUM(lineitem.l_quantity) -------------LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey ---------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] ---------------SubqueryAlias: __correlated_sq_2 -----------------Projection: part.p_partkey -------------------Filter: part.p_name LIKE Utf8("forest%") ---------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("forest%")] -------------SubqueryAlias: __scalar_sq_3 ---------------Projection: Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey -----------------Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] -------------------Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity ---------------------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") -----------------------TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131")] +01)Sort: supplier.s_name ASC NULLS LAST +02)--Projection: supplier.s_name, supplier.s_address +03)----LeftSemi Join: supplier.s_suppkey = __correlated_sq_1.ps_suppkey +04)------Projection: supplier.s_suppkey, supplier.s_name, supplier.s_address +05)--------Inner Join: supplier.s_nationkey = nation.n_nationkey +06)----------TableScan: supplier projection=[s_suppkey, s_name, s_address, s_nationkey] +07)----------Projection: nation.n_nationkey +08)------------Filter: nation.n_name = Utf8("CANADA") +09)--------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("CANADA")] +10)------SubqueryAlias: __correlated_sq_1 +11)--------Projection: partsupp.ps_suppkey +12)----------Inner Join: partsupp.ps_partkey = __scalar_sq_3.l_partkey, partsupp.ps_suppkey = __scalar_sq_3.l_suppkey Filter: CAST(partsupp.ps_availqty AS Float64) > __scalar_sq_3.Float64(0.5) * SUM(lineitem.l_quantity) +13)------------LeftSemi Join: partsupp.ps_partkey = __correlated_sq_2.p_partkey +14)--------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_availqty] +15)--------------SubqueryAlias: __correlated_sq_2 +16)----------------Projection: part.p_partkey +17)------------------Filter: part.p_name LIKE Utf8("forest%") +18)--------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("forest%")] +19)------------SubqueryAlias: __scalar_sq_3 +20)--------------Projection: Float64(0.5) * CAST(SUM(lineitem.l_quantity) AS Float64), lineitem.l_partkey, lineitem.l_suppkey +21)----------------Aggregate: groupBy=[[lineitem.l_partkey, lineitem.l_suppkey]], aggr=[[SUM(lineitem.l_quantity)]] +22)------------------Projection: lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity +23)--------------------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") +24)----------------------TableScan: lineitem projection=[l_partkey, l_suppkey, l_quantity, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131")] physical_plan -SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] ---SortExec: expr=[s_name@0 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=8192 -------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------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 -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -------------------------FilterExec: n_name@1 = CANADA ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -------------CoalesceBatchesExec: target_batch_size=8192 ---------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -----------------------------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 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ---------------------------------FilterExec: p_name@1 LIKE forest% -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 ---------------------------------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 +01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] +02)--SortExec: expr=[s_name@0 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------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 +20)--------CoalesceBatchesExec: target_batch_size=8192 +21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +22)------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +24)----------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +26)--------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +28)------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +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)----------------------------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 +38)----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +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)--------------------------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 a13d57beb16c..3d6428882374 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -58,93 +58,93 @@ order by s_name; ---- logical_plan -Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST ---Projection: supplier.s_name, COUNT(*) AS numwait -----Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -------Projection: supplier.s_name ---------LeftAnti Join: l1.l_orderkey = __correlated_sq_2.l_orderkey Filter: __correlated_sq_2.l_suppkey != l1.l_suppkey -----------LeftSemi Join: l1.l_orderkey = __correlated_sq_1.l_orderkey Filter: __correlated_sq_1.l_suppkey != l1.l_suppkey -------------Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey ---------------Inner Join: supplier.s_nationkey = nation.n_nationkey -----------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey -------------------Inner Join: l1.l_orderkey = orders.o_orderkey ---------------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey -----------------------Inner Join: supplier.s_suppkey = l1.l_suppkey -------------------------TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] -------------------------SubqueryAlias: l1 ---------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey -----------------------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] ---------------------Projection: orders.o_orderkey -----------------------Filter: orders.o_orderstatus = Utf8("F") -------------------------TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8("F")] -----------------Projection: nation.n_nationkey -------------------Filter: nation.n_name = Utf8("SAUDI ARABIA") ---------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("SAUDI ARABIA")] -------------SubqueryAlias: __correlated_sq_1 ---------------SubqueryAlias: l2 -----------------TableScan: lineitem projection=[l_orderkey, l_suppkey] -----------SubqueryAlias: __correlated_sq_2 -------------SubqueryAlias: l3 ---------------Projection: lineitem.l_orderkey, lineitem.l_suppkey -----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +01)Sort: numwait DESC NULLS FIRST, supplier.s_name ASC NULLS LAST +02)--Projection: supplier.s_name, COUNT(*) AS numwait +03)----Aggregate: groupBy=[[supplier.s_name]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------Projection: supplier.s_name +05)--------LeftAnti Join: l1.l_orderkey = __correlated_sq_2.l_orderkey Filter: __correlated_sq_2.l_suppkey != l1.l_suppkey +06)----------LeftSemi Join: l1.l_orderkey = __correlated_sq_1.l_orderkey Filter: __correlated_sq_1.l_suppkey != l1.l_suppkey +07)------------Projection: supplier.s_name, l1.l_orderkey, l1.l_suppkey +08)--------------Inner Join: supplier.s_nationkey = nation.n_nationkey +09)----------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey +10)------------------Inner Join: l1.l_orderkey = orders.o_orderkey +11)--------------------Projection: supplier.s_name, supplier.s_nationkey, l1.l_orderkey, l1.l_suppkey +12)----------------------Inner Join: supplier.s_suppkey = l1.l_suppkey +13)------------------------TableScan: supplier projection=[s_suppkey, s_name, s_nationkey] +14)------------------------SubqueryAlias: l1 +15)--------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey +16)----------------------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate +17)------------------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +18)--------------------Projection: orders.o_orderkey +19)----------------------Filter: orders.o_orderstatus = Utf8("F") +20)------------------------TableScan: orders projection=[o_orderkey, o_orderstatus], partial_filters=[orders.o_orderstatus = Utf8("F")] +21)----------------Projection: nation.n_nationkey +22)------------------Filter: nation.n_name = Utf8("SAUDI ARABIA") +23)--------------------TableScan: nation projection=[n_nationkey, n_name], partial_filters=[nation.n_name = Utf8("SAUDI ARABIA")] +24)------------SubqueryAlias: __correlated_sq_1 +25)--------------SubqueryAlias: l2 +26)----------------TableScan: lineitem projection=[l_orderkey, l_suppkey] +27)----------SubqueryAlias: __correlated_sq_2 +28)------------SubqueryAlias: l3 +29)--------------Projection: lineitem.l_orderkey, lineitem.l_suppkey +30)----------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate +31)------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan -SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] ---SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST] -----ProjectionExec: expr=[s_name@0 as s_name, COUNT(*)@1 as numwait] -------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(*)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------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 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 ---------------------------------------------------------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 ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] -----------------------------------------------FilterExec: o_orderstatus@1 = F -------------------------------------------------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 -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] ---------------------------------------FilterExec: n_name@1 = SAUDI ARABIA -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------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 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] ---------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -----------------------------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 +01)SortPreservingMergeExec: [numwait@1 DESC,s_name@0 ASC NULLS LAST] +02)--SortExec: expr=[numwait@1 DESC,s_name@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[s_name@0 as s_name, COUNT(*)@1 as numwait] +04)------AggregateExec: mode=FinalPartitioned, gby=[s_name@0 as s_name], aggr=[COUNT(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------------------------------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)------------------------------------------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)----------------------------------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 +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +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)----------------------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 d1b19d400572..9aecac8783db 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -56,52 +56,52 @@ order by cntrycode; ---- logical_plan -Sort: custsale.cntrycode ASC NULLS LAST ---Projection: custsale.cntrycode, COUNT(*) AS numcust, SUM(custsale.c_acctbal) AS totacctbal -----Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[COUNT(Int64(1)) AS COUNT(*), SUM(custsale.c_acctbal)]] -------SubqueryAlias: custsale ---------Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal -----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.AVG(customer.c_acctbal) -------------Projection: customer.c_phone, customer.c_acctbal ---------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey -----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) -------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])] -----------------SubqueryAlias: __correlated_sq_1 -------------------TableScan: orders projection=[o_custkey] -------------SubqueryAlias: __scalar_sq_2 ---------------Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] -----------------Projection: customer.c_acctbal -------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) ---------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2) AS customer.c_acctbal > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]), customer.c_acctbal > Decimal128(Some(0),15,2)] +01)Sort: custsale.cntrycode ASC NULLS LAST +02)--Projection: custsale.cntrycode, COUNT(*) AS numcust, SUM(custsale.c_acctbal) AS totacctbal +03)----Aggregate: groupBy=[[custsale.cntrycode]], aggr=[[COUNT(Int64(1)) AS COUNT(*), SUM(custsale.c_acctbal)]] +04)------SubqueryAlias: custsale +05)--------Projection: substr(customer.c_phone, Int64(1), Int64(2)) AS cntrycode, customer.c_acctbal +06)----------Inner Join: Filter: CAST(customer.c_acctbal AS Decimal128(19, 6)) > __scalar_sq_2.AVG(customer.c_acctbal) +07)------------Projection: customer.c_phone, customer.c_acctbal +08)--------------LeftAnti Join: customer.c_custkey = __correlated_sq_1.o_custkey +09)----------------Filter: substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) +10)------------------TableScan: customer projection=[c_custkey, c_phone, c_acctbal], partial_filters=[substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")])] +11)----------------SubqueryAlias: __correlated_sq_1 +12)------------------TableScan: orders projection=[o_custkey] +13)------------SubqueryAlias: __scalar_sq_2 +14)--------------Aggregate: groupBy=[[]], aggr=[[AVG(customer.c_acctbal)]] +15)----------------Projection: customer.c_acctbal +16)------------------Filter: customer.c_acctbal > Decimal128(Some(0),15,2) AND substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]) +17)--------------------TableScan: customer projection=[c_phone, c_acctbal], partial_filters=[customer.c_acctbal > Decimal128(Some(0),15,2) AS customer.c_acctbal > Decimal128(Some(0),30,15), substr(customer.c_phone, Int64(1), Int64(2)) IN ([Utf8("13"), Utf8("31"), Utf8("23"), Utf8("29"), Utf8("30"), Utf8("18"), Utf8("17")]), customer.c_acctbal > Decimal128(Some(0),15,2)] physical_plan -SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] ---SortExec: expr=[cntrycode@0 ASC NULLS LAST] -----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(*)@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] -------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] ---------------ProjectionExec: expr=[substr(c_phone@0, 1, 2) as cntrycode, c_acctbal@1 as c_acctbal] -----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------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") }]) -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------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 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 ---------------------------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 -------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] ---------------------CoalescePartitionsExec -----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] -----------------------------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") }]) -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +01)SortPreservingMergeExec: [cntrycode@0 ASC NULLS LAST] +02)--SortExec: expr=[cntrycode@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[cntrycode@0 as cntrycode, COUNT(*)@1 as numcust, SUM(custsale.c_acctbal)@2 as totacctbal] +04)------AggregateExec: mode=FinalPartitioned, gby=[cntrycode@0 as cntrycode], aggr=[COUNT(*), SUM(custsale.c_acctbal)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([cntrycode@0], 4), input_partitions=4 +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)----------------NestedLoopJoinExec: join_type=Inner, filter=CAST(c_acctbal@0 AS Decimal128(19, 6)) > AVG(customer.c_acctbal)@1 +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)], projection=[c_phone@1, c_acctbal@2] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------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") }]) +16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------------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 +18)----------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +20)--------------------------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 +21)------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] +22)--------------------CoalescePartitionsExec +23)----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] +24)------------------------CoalesceBatchesExec: target_batch_size=8192 +25)--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] +26)----------------------------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") }]) +27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)--------------------------------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 93b1737052f6..bb20789f35c0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -42,55 +42,55 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=10 -----Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority -------Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] ---------Projection: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount -----------Inner Join: orders.o_orderkey = lineitem.l_orderkey -------------Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority ---------------Inner Join: customer.c_custkey = orders.o_custkey -----------------Projection: customer.c_custkey -------------------Filter: customer.c_mktsegment = Utf8("BUILDING") ---------------------TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8("BUILDING")] -----------------Filter: orders.o_orderdate < Date32("9204") -------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("9204")] -------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount ---------------Filter: lineitem.l_shipdate > Date32("9204") -----------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] +01)Limit: skip=0, fetch=10 +02)--Sort: revenue DESC NULLS FIRST, orders.o_orderdate ASC NULLS LAST, fetch=10 +03)----Projection: lineitem.l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue, orders.o_orderdate, orders.o_shippriority +04)------Aggregate: groupBy=[[lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +05)--------Projection: orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +06)----------Inner Join: orders.o_orderkey = lineitem.l_orderkey +07)------------Projection: orders.o_orderkey, orders.o_orderdate, orders.o_shippriority +08)--------------Inner Join: customer.c_custkey = orders.o_custkey +09)----------------Projection: customer.c_custkey +10)------------------Filter: customer.c_mktsegment = Utf8("BUILDING") +11)--------------------TableScan: customer projection=[c_custkey, c_mktsegment], partial_filters=[customer.c_mktsegment = Utf8("BUILDING")] +12)----------------Filter: orders.o_orderdate < Date32("9204") +13)------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate, o_shippriority], partial_filters=[orders.o_orderdate < Date32("9204")] +14)------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount +15)--------------Filter: lineitem.l_shipdate > Date32("9204") +16)----------------TableScan: lineitem projection=[l_orderkey, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate > Date32("9204")] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 -----SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] -------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] ---------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -------------------------------------FilterExec: c_mktsegment@1 = BUILDING ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------FilterExec: o_orderdate@2 < 9204 -------------------------------------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 ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -----------------------------FilterExec: l_shipdate@3 > 9204 -------------------------------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 +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [revenue@1 DESC,o_orderdate@2 ASC NULLS LAST], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[revenue@1 DESC,o_orderdate@2 ASC NULLS LAST] +04)------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@3 as revenue, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority] +05)--------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey, o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[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)--------------------------------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 +22)----------------------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +24)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)----------------------------------FilterExec: o_orderdate@2 < 9204 +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)------------------------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 query IRDI select diff --git a/datafusion/sqllogictest/test_files/tpch/q4.slt.part b/datafusion/sqllogictest/test_files/tpch/q4.slt.part index ff70f2a652c1..929b5cbefa8c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -40,40 +40,40 @@ order by o_orderpriority; ---- logical_plan -Sort: orders.o_orderpriority ASC NULLS LAST ---Projection: orders.o_orderpriority, COUNT(*) AS order_count -----Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -------Projection: orders.o_orderpriority ---------LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey -----------Projection: orders.o_orderkey, orders.o_orderpriority -------------Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") ---------------TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("8582"), orders.o_orderdate < Date32("8674")] -----------SubqueryAlias: __correlated_sq_1 -------------Projection: lineitem.l_orderkey ---------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate -----------------TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] +01)Sort: orders.o_orderpriority ASC NULLS LAST +02)--Projection: orders.o_orderpriority, COUNT(*) AS order_count +03)----Aggregate: groupBy=[[orders.o_orderpriority]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +04)------Projection: orders.o_orderpriority +05)--------LeftSemi Join: orders.o_orderkey = __correlated_sq_1.l_orderkey +06)----------Projection: orders.o_orderkey, orders.o_orderpriority +07)------------Filter: orders.o_orderdate >= Date32("8582") AND orders.o_orderdate < Date32("8674") +08)--------------TableScan: orders projection=[o_orderkey, o_orderdate, o_orderpriority], partial_filters=[orders.o_orderdate >= Date32("8582"), orders.o_orderdate < Date32("8674")] +09)----------SubqueryAlias: __correlated_sq_1 +10)------------Projection: lineitem.l_orderkey +11)--------------Filter: lineitem.l_receiptdate > lineitem.l_commitdate +12)----------------TableScan: lineitem projection=[l_orderkey, l_commitdate, l_receiptdate], partial_filters=[lineitem.l_receiptdate > lineitem.l_commitdate] physical_plan -SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] ---SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] -----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(*)@1 as order_count] -------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderpriority@1] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] ---------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 -----------------------------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 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] ---------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 -----------------------------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 +01)SortPreservingMergeExec: [o_orderpriority@0 ASC NULLS LAST] +02)--SortExec: expr=[o_orderpriority@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[o_orderpriority@0 as o_orderpriority, COUNT(*)@1 as order_count] +04)------AggregateExec: mode=FinalPartitioned, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +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)----------------------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)----------------------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 c99863a2c5b8..7744a2b6038d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -43,83 +43,83 @@ order by revenue desc; ---- logical_plan -Sort: revenue DESC NULLS FIRST ---Projection: nation.n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue -----Aggregate: groupBy=[[nation.n_name]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] -------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name ---------Inner Join: nation.n_regionkey = region.r_regionkey -----------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey -------------Inner Join: supplier.s_nationkey = nation.n_nationkey ---------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey -----------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey -------------------Projection: customer.c_nationkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount ---------------------Inner Join: orders.o_orderkey = lineitem.l_orderkey -----------------------Projection: customer.c_nationkey, orders.o_orderkey -------------------------Inner Join: customer.c_custkey = orders.o_custkey ---------------------------TableScan: customer projection=[c_custkey, c_nationkey] ---------------------------Projection: orders.o_orderkey, orders.o_custkey -----------------------------Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") -------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8766"), orders.o_orderdate < Date32("9131")] -----------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] -------------------TableScan: supplier projection=[s_suppkey, s_nationkey] ---------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] -----------Projection: region.r_regionkey -------------Filter: region.r_name = Utf8("ASIA") ---------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("ASIA")] +01)Sort: revenue DESC NULLS FIRST +02)--Projection: nation.n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount) AS revenue +03)----Aggregate: groupBy=[[nation.n_name]], aggr=[[SUM(lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount)) AS SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)]] +04)------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name +05)--------Inner Join: nation.n_regionkey = region.r_regionkey +06)----------Projection: lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey +07)------------Inner Join: supplier.s_nationkey = nation.n_nationkey +08)--------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey +09)----------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey, customer.c_nationkey = supplier.s_nationkey +10)------------------Projection: customer.c_nationkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +11)--------------------Inner Join: orders.o_orderkey = lineitem.l_orderkey +12)----------------------Projection: customer.c_nationkey, orders.o_orderkey +13)------------------------Inner Join: customer.c_custkey = orders.o_custkey +14)--------------------------TableScan: customer projection=[c_custkey, c_nationkey] +15)--------------------------Projection: orders.o_orderkey, orders.o_custkey +16)----------------------------Filter: orders.o_orderdate >= Date32("8766") AND orders.o_orderdate < Date32("9131") +17)------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("8766"), orders.o_orderdate < Date32("9131")] +18)----------------------TableScan: lineitem projection=[l_orderkey, l_suppkey, l_extendedprice, l_discount] +19)------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +20)--------------TableScan: nation projection=[n_nationkey, n_name, n_regionkey] +21)----------Projection: region.r_regionkey +22)------------Filter: region.r_name = Utf8("ASIA") +23)--------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("ASIA")] physical_plan -SortPreservingMergeExec: [revenue@1 DESC] ---SortExec: expr=[revenue@1 DESC] -----ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] -------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -----------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 -------------------------------------------------------------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 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------------------------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_extendedprice, l_discount], has_header=false -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 ---------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------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 -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] ---------------------------FilterExec: r_name@1 = ASIA -----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +01)SortPreservingMergeExec: [revenue@1 DESC] +02)--SortExec: expr=[revenue@1 DESC] +03)----ProjectionExec: expr=[n_name@0 as n_name, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as revenue] +04)------AggregateExec: mode=FinalPartitioned, gby=[n_name@0 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)------------------------------------------------------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 +37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +38)----------------------------------------------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_extendedprice, l_discount], has_header=false +39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +43)--------------------------CoalesceBatchesExec: target_batch_size=8192 +44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +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, n_regionkey], has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +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 983ee5082453..f3992145a8c1 100644 --- a/datafusion/sqllogictest/test_files/tpch/q6.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q6.slt.part @@ -28,20 +28,20 @@ where and l_quantity < 24; ---- logical_plan -Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue ---Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] -----Projection: lineitem.l_extendedprice, lineitem.l_discount -------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) ---------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131"), lineitem.l_discount >= Decimal128(Some(5),15,2), lineitem.l_discount <= Decimal128(Some(7),15,2), lineitem.l_quantity < Decimal128(Some(2400),15,2)] +01)Projection: SUM(lineitem.l_extendedprice * lineitem.l_discount) AS revenue +02)--Aggregate: groupBy=[[]], aggr=[[SUM(lineitem.l_extendedprice * lineitem.l_discount)]] +03)----Projection: lineitem.l_extendedprice, lineitem.l_discount +04)------Filter: lineitem.l_shipdate >= Date32("8766") AND lineitem.l_shipdate < Date32("9131") AND lineitem.l_discount >= Decimal128(Some(5),15,2) AND lineitem.l_discount <= Decimal128(Some(7),15,2) AND lineitem.l_quantity < Decimal128(Some(2400),15,2) +05)--------TableScan: lineitem projection=[l_quantity, l_extendedprice, l_discount, l_shipdate], partial_filters=[lineitem.l_shipdate >= Date32("8766"), lineitem.l_shipdate < Date32("9131"), lineitem.l_discount >= Decimal128(Some(5),15,2), lineitem.l_discount <= Decimal128(Some(7),15,2), lineitem.l_quantity < Decimal128(Some(2400),15,2)] physical_plan -ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] ---AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -------------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 ---------------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 +01)ProjectionExec: expr=[SUM(lineitem.l_extendedprice * lineitem.l_discount)@0 as revenue] +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)--------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 02016a444355..1da304b3f946 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -56,112 +56,112 @@ order by o_year; ---- logical_plan -Sort: all_nations.o_year ASC NULLS LAST ---Projection: all_nations.o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END) AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume) AS Decimal128(12, 2)) AS Decimal128(15, 2)) AS mkt_share -----Aggregate: groupBy=[[all_nations.o_year]], aggr=[[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]] -------SubqueryAlias: all_nations ---------Projection: date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume, n2.n_name AS nation -----------Inner Join: n1.n_regionkey = region.r_regionkey -------------Projection: lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, n1.n_regionkey, n2.n_name ---------------Inner Join: supplier.s_nationkey = n2.n_nationkey -----------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, n1.n_regionkey -------------------Inner Join: customer.c_nationkey = n1.n_nationkey ---------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, customer.c_nationkey -----------------------Inner Join: orders.o_custkey = customer.c_custkey -------------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_custkey, orders.o_orderdate ---------------------------Inner Join: lineitem.l_orderkey = orders.o_orderkey -----------------------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey -------------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey ---------------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount -----------------------------------Inner Join: part.p_partkey = lineitem.l_partkey -------------------------------------Projection: part.p_partkey ---------------------------------------Filter: part.p_type = Utf8("ECONOMY ANODIZED STEEL") -----------------------------------------TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8("ECONOMY ANODIZED STEEL")] -------------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] ---------------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------------Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") -------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("9131"), orders.o_orderdate <= Date32("9861")] -------------------------TableScan: customer projection=[c_custkey, c_nationkey] ---------------------SubqueryAlias: n1 -----------------------TableScan: nation projection=[n_nationkey, n_regionkey] -----------------SubqueryAlias: n2 -------------------TableScan: nation projection=[n_nationkey, n_name] -------------Projection: region.r_regionkey ---------------Filter: region.r_name = Utf8("AMERICA") -----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("AMERICA")] +01)Sort: all_nations.o_year ASC NULLS LAST +02)--Projection: all_nations.o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END) AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume) AS Decimal128(12, 2)) AS Decimal128(15, 2)) AS mkt_share +03)----Aggregate: groupBy=[[all_nations.o_year]], aggr=[[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Decimal128(Some(0),38,4) END) AS SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)]] +04)------SubqueryAlias: all_nations +05)--------Projection: date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) AS volume, n2.n_name AS nation +06)----------Inner Join: n1.n_regionkey = region.r_regionkey +07)------------Projection: lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, n1.n_regionkey, n2.n_name +08)--------------Inner Join: supplier.s_nationkey = n2.n_nationkey +09)----------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, n1.n_regionkey +10)------------------Inner Join: customer.c_nationkey = n1.n_nationkey +11)--------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderdate, customer.c_nationkey +12)----------------------Inner Join: orders.o_custkey = customer.c_custkey +13)------------------------Projection: lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_custkey, orders.o_orderdate +14)--------------------------Inner Join: lineitem.l_orderkey = orders.o_orderkey +15)----------------------------Projection: lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey +16)------------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey +17)--------------------------------Projection: lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount +18)----------------------------------Inner Join: part.p_partkey = lineitem.l_partkey +19)------------------------------------Projection: part.p_partkey +20)--------------------------------------Filter: part.p_type = Utf8("ECONOMY ANODIZED STEEL") +21)----------------------------------------TableScan: part projection=[p_partkey, p_type], partial_filters=[part.p_type = Utf8("ECONOMY ANODIZED STEEL")] +22)------------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_extendedprice, l_discount] +23)--------------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +24)----------------------------Filter: orders.o_orderdate >= Date32("9131") AND orders.o_orderdate <= Date32("9861") +25)------------------------------TableScan: orders projection=[o_orderkey, o_custkey, o_orderdate], partial_filters=[orders.o_orderdate >= Date32("9131"), orders.o_orderdate <= Date32("9861")] +26)------------------------TableScan: customer projection=[c_custkey, c_nationkey] +27)--------------------SubqueryAlias: n1 +28)----------------------TableScan: nation projection=[n_nationkey, n_regionkey] +29)----------------SubqueryAlias: n2 +30)------------------TableScan: nation projection=[n_nationkey, n_name] +31)------------Projection: region.r_regionkey +32)--------------Filter: region.r_name = Utf8("AMERICA") +33)----------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("AMERICA")] physical_plan -SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] ---SortExec: expr=[o_year@0 ASC NULLS LAST] -----ProjectionExec: expr=[o_year@0 as o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume)@2 AS Decimal128(12, 2)) AS Decimal128(15, 2)) as mkt_share] -------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] ---------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 ---------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 -----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 -------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------------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] ---------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -----------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL -------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false ---------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 ---------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 -------------------------------------------------------------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 ---------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -----------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -----------------------------FilterExec: r_name@1 = AMERICA -------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +01)SortPreservingMergeExec: [o_year@0 ASC NULLS LAST] +02)--SortExec: expr=[o_year@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[o_year@0 as o_year, CAST(CAST(SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END)@1 AS Decimal128(12, 2)) / CAST(SUM(all_nations.volume)@2 AS Decimal128(12, 2)) AS Decimal128(15, 2)) as mkt_share] +04)------AggregateExec: mode=FinalPartitioned, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] +08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] +31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +33)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] +35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +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 +42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +44)------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 +53)------------------------------------------------------------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 +54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +62)----------------------------CoalesceBatchesExec: target_batch_size=8192 +63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)------------------------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 query RR select diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 48e7b236ac4a..e3e6a4d0e0d7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -52,81 +52,81 @@ order by limit 10; ---- logical_plan -Limit: skip=0, fetch=10 ---Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST, fetch=10 -----Projection: profit.nation, profit.o_year, SUM(profit.amount) AS sum_profit -------Aggregate: groupBy=[[profit.nation, profit.o_year]], aggr=[[SUM(profit.amount)]] ---------SubqueryAlias: profit -----------Projection: nation.n_name AS nation, date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) - partsupp.ps_supplycost * lineitem.l_quantity AS amount -------------Inner Join: supplier.s_nationkey = nation.n_nationkey ---------------Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost, orders.o_orderdate -----------------Inner Join: lineitem.l_orderkey = orders.o_orderkey -------------------Projection: lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost ---------------------Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey -----------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey -------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey ---------------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount -----------------------------Inner Join: part.p_partkey = lineitem.l_partkey -------------------------------Projection: part.p_partkey ---------------------------------Filter: part.p_name LIKE Utf8("%green%") -----------------------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("%green%")] -------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] ---------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] -----------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] -------------------TableScan: orders projection=[o_orderkey, o_orderdate] ---------------TableScan: nation projection=[n_nationkey, n_name] +01)Limit: skip=0, fetch=10 +02)--Sort: profit.nation ASC NULLS LAST, profit.o_year DESC NULLS FIRST, fetch=10 +03)----Projection: profit.nation, profit.o_year, SUM(profit.amount) AS sum_profit +04)------Aggregate: groupBy=[[profit.nation, profit.o_year]], aggr=[[SUM(profit.amount)]] +05)--------SubqueryAlias: profit +06)----------Projection: nation.n_name AS nation, date_part(Utf8("YEAR"), orders.o_orderdate) AS o_year, lineitem.l_extendedprice * (Decimal128(Some(1),20,0) - lineitem.l_discount) - partsupp.ps_supplycost * lineitem.l_quantity AS amount +07)------------Inner Join: supplier.s_nationkey = nation.n_nationkey +08)--------------Projection: lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost, orders.o_orderdate +09)----------------Inner Join: lineitem.l_orderkey = orders.o_orderkey +10)------------------Projection: lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, partsupp.ps_supplycost +11)--------------------Inner Join: lineitem.l_suppkey = partsupp.ps_suppkey, lineitem.l_partkey = partsupp.ps_partkey +12)----------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey +13)------------------------Inner Join: lineitem.l_suppkey = supplier.s_suppkey +14)--------------------------Projection: lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount +15)----------------------------Inner Join: part.p_partkey = lineitem.l_partkey +16)------------------------------Projection: part.p_partkey +17)--------------------------------Filter: part.p_name LIKE Utf8("%green%") +18)----------------------------------TableScan: part projection=[p_partkey, p_name], partial_filters=[part.p_name LIKE Utf8("%green%")] +19)------------------------------TableScan: lineitem projection=[l_orderkey, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount] +20)--------------------------TableScan: supplier projection=[s_suppkey, s_nationkey] +21)----------------------TableScan: partsupp projection=[ps_partkey, ps_suppkey, ps_supplycost] +22)------------------TableScan: orders projection=[o_orderkey, o_orderdate] +23)--------------TableScan: nation projection=[n_nationkey, n_name] physical_plan -GlobalLimitExec: skip=0, fetch=10 ---SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 -----SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] -------ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] ---------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -----------CoalesceBatchesExec: target_batch_size=8192 -------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 ---------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 ---------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -----------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 -------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 ---------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------------------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] -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] ---------------------------------------------------------------FilterExec: p_name@1 LIKE %green% -----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -----------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 ---------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false ---------------------------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -------------------------------------------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 -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -----------------------------------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], has_header=false -----------------------CoalesceBatchesExec: target_batch_size=8192 -------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [nation@0 ASC NULLS LAST,o_year@1 DESC], fetch=10 +03)----SortExec: TopK(fetch=10), expr=[nation@0 ASC NULLS LAST,o_year@1 DESC] +04)------ProjectionExec: expr=[nation@0 as nation, o_year@1 as o_year, SUM(profit.amount)@2 as sum_profit] +05)--------AggregateExec: mode=FinalPartitioned, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 +08)--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] +09)----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +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)----------------------------------------------------------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 +35)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +37)----------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +38)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@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/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +42)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +44)------------------------------------------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 +45)------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +47)----------------------------------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], has_header=false +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false query TRR select diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 003e614e6962..550aebef0bcd 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -276,36 +276,36 @@ logical_plan 09)--------TableScan: t2 projection=[id, name] 10)------TableScan: t1 projection=[id, name] physical_plan -UnionExec ---CoalesceBatchesExec: target_batch_size=2 -----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(id@0, CAST(t2.id AS Int32)@1), (name@1, name@0)] -------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] ---------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] -------CoalesceBatchesExec: target_batch_size=2 ---------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@1, name@0], 4), input_partitions=4 -----------ProjectionExec: expr=[name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -----CoalesceBatchesExec: target_batch_size=2 -------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] ---------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -----------------CoalesceBatchesExec: target_batch_size=2 -------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 ---------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[1] ---------CoalesceBatchesExec: target_batch_size=2 -----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)UnionExec +02)--CoalesceBatchesExec: target_batch_size=2 +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 +07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +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)@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] +16)----CoalesceBatchesExec: target_batch_size=2 +17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +18)--------CoalesceBatchesExec: target_batch_size=2 +19)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +20)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +21)--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +22)----------------CoalesceBatchesExec: target_batch_size=2 +23)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +24)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +25)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)------------------------MemoryExec: partitions=1, partition_sizes=[1] +27)--------CoalesceBatchesExec: target_batch_size=2 +28)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +29)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 6d7953e38f3c..59b816c317c0 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -253,39 +253,39 @@ WITH _sample_data AS ( ORDER BY d.b; ---- logical_plan -Sort: d.b ASC NULLS LAST ---Projection: d.b, MAX(d.a) AS max_a -----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a)]] -------SubqueryAlias: d ---------SubqueryAlias: _data2 -----------SubqueryAlias: s -------------SubqueryAlias: _sample_data ---------------Union -----------------Projection: Int64(1) AS a, Utf8("aa") AS b -------------------EmptyRelation -----------------Projection: Int64(3) AS a, Utf8("aa") AS b -------------------EmptyRelation -----------------Projection: Int64(5) AS a, Utf8("bb") AS b -------------------EmptyRelation -----------------Projection: Int64(7) AS a, Utf8("bb") AS b -------------------EmptyRelation +01)Sort: d.b ASC NULLS LAST +02)--Projection: d.b, MAX(d.a) AS max_a +03)----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a)]] +04)------SubqueryAlias: d +05)--------SubqueryAlias: _data2 +06)----------SubqueryAlias: s +07)------------SubqueryAlias: _sample_data +08)--------------Union +09)----------------Projection: Int64(1) AS a, Utf8("aa") AS b +10)------------------EmptyRelation +11)----------------Projection: Int64(3) AS a, Utf8("aa") AS b +12)------------------EmptyRelation +13)----------------Projection: Int64(5) AS a, Utf8("bb") AS b +14)------------------EmptyRelation +15)----------------Projection: Int64(7) AS a, Utf8("bb") AS b +16)------------------EmptyRelation physical_plan -SortPreservingMergeExec: [b@0 ASC NULLS LAST] ---SortExec: expr=[b@0 ASC NULLS LAST] -----ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a] -------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] ---------CoalesceBatchesExec: target_batch_size=8192 -----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] ---------------UnionExec -----------------ProjectionExec: expr=[1 as a, aa as b] -------------------PlaceholderRowExec -----------------ProjectionExec: expr=[3 as a, aa as b] -------------------PlaceholderRowExec -----------------ProjectionExec: expr=[5 as a, bb as b] -------------------PlaceholderRowExec -----------------ProjectionExec: expr=[7 as a, bb as b] -------------------PlaceholderRowExec +01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] +02)--SortExec: expr=[b@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a] +04)------AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[MAX(d.a)] +08)--------------UnionExec +09)----------------ProjectionExec: expr=[1 as a, aa as b] +10)------------------PlaceholderRowExec +11)----------------ProjectionExec: expr=[3 as a, aa as b] +12)------------------PlaceholderRowExec +13)----------------ProjectionExec: expr=[5 as a, bb as b] +14)------------------PlaceholderRowExec +15)----------------ProjectionExec: expr=[7 as a, bb as b] +16)------------------PlaceholderRowExec # Check actual result: query TI @@ -336,42 +336,42 @@ WITH _sample_data AS ( ORDER BY d.b ---- logical_plan -Sort: d.b ASC NULLS LAST ---Projection: d.b, MAX(d.a) AS max_a, MAX(d.seq) -----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a), MAX(d.seq)]] -------SubqueryAlias: d ---------SubqueryAlias: _data2 -----------Projection: ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS seq, s.a, s.b -------------WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------------SubqueryAlias: s -----------------SubqueryAlias: _sample_data -------------------Union ---------------------Projection: Int64(1) AS a, Utf8("aa") AS b -----------------------EmptyRelation ---------------------Projection: Int64(3) AS a, Utf8("aa") AS b -----------------------EmptyRelation ---------------------Projection: Int64(5) AS a, Utf8("bb") AS b -----------------------EmptyRelation ---------------------Projection: Int64(7) AS a, Utf8("bb") AS b -----------------------EmptyRelation +01)Sort: d.b ASC NULLS LAST +02)--Projection: d.b, MAX(d.a) AS max_a, MAX(d.seq) +03)----Aggregate: groupBy=[[d.b]], aggr=[[MAX(d.a), MAX(d.seq)]] +04)------SubqueryAlias: d +05)--------SubqueryAlias: _data2 +06)----------Projection: ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS seq, s.a, s.b +07)------------WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +08)--------------SubqueryAlias: s +09)----------------SubqueryAlias: _sample_data +10)------------------Union +11)--------------------Projection: Int64(1) AS a, Utf8("aa") AS b +12)----------------------EmptyRelation +13)--------------------Projection: Int64(3) AS a, Utf8("aa") AS b +14)----------------------EmptyRelation +15)--------------------Projection: Int64(5) AS a, Utf8("bb") AS b +16)----------------------EmptyRelation +17)--------------------Projection: Int64(7) AS a, Utf8("bb") AS b +18)----------------------EmptyRelation physical_plan -SortPreservingMergeExec: [b@0 ASC NULLS LAST] ---ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] -----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted -------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] -------------CoalesceBatchesExec: target_batch_size=8192 ---------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 -----------------UnionExec -------------------ProjectionExec: expr=[1 as a, aa as b] ---------------------PlaceholderRowExec -------------------ProjectionExec: expr=[3 as a, aa as b] ---------------------PlaceholderRowExec -------------------ProjectionExec: expr=[5 as a, bb as b] ---------------------PlaceholderRowExec -------------------ProjectionExec: expr=[7 as a, bb as b] ---------------------PlaceholderRowExec +01)SortPreservingMergeExec: [b@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] +03)----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted +04)------ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] +05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------RepartitionExec: partitioning=Hash([b@1], 4), input_partitions=4 +09)----------------UnionExec +10)------------------ProjectionExec: expr=[1 as a, aa as b] +11)--------------------PlaceholderRowExec +12)------------------ProjectionExec: expr=[3 as a, aa as b] +13)--------------------PlaceholderRowExec +14)------------------ProjectionExec: expr=[5 as a, bb as b] +15)--------------------PlaceholderRowExec +16)------------------ProjectionExec: expr=[7 as a, bb as b] +17)--------------------PlaceholderRowExec # check actual result @@ -1202,18 +1202,18 @@ EXPLAIN SELECT FROM aggregate_test_100 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 ---WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: aggregate_test_100 projection=[c8, c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 +02)--WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: aggregate_test_100 projection=[c8, c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum2] ---BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum2] +02)--BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c8, c9], has_header=true # over_order_by_sort_keys_sorting_prefix_compacting @@ -1223,18 +1223,18 @@ query TT EXPLAIN SELECT c2, MAX(c9) OVER (ORDER BY c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100 ---- logical_plan -Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: aggregate_test_100 projection=[c2, c9] +01)Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan -ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [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(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true +01)ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +03)----BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [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(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true # FIXME: for now we are not detecting prefix of sorting keys in order to re-arrange with global and save one SortExec @@ -1245,21 +1245,21 @@ query TT EXPLAIN SELECT c2, MAX(c9) OVER (ORDER BY c9, c2), SUM(c9) OVER (), MIN(c9) OVER (ORDER BY c2, c9) from aggregate_test_100 ORDER BY c2 ---- logical_plan -Sort: aggregate_test_100.c2 ASC NULLS LAST ---Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c2, c9] +01)Sort: aggregate_test_100.c2 ASC NULLS LAST +02)--Projection: aggregate_test_100.c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c2, c9] physical_plan -SortExec: expr=[c2@0 ASC NULLS LAST] ---ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] -------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true +01)SortExec: expr=[c2@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING, MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +03)----WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED 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(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }] +04)------BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST] +06)----------BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 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: Preceding(Int8(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c9], has_header=true # test_window_partition_by_order_by statement ok @@ -1272,24 +1272,24 @@ EXPLAIN SELECT FROM aggregate_test_100 ---- logical_plan -Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING ---WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] -----Projection: aggregate_test_100.c1, aggregate_test_100.c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING -------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c1, c2, c4] +01)Projection: SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +02)--WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +03)----Projection: aggregate_test_100.c1, aggregate_test_100.c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +04)------WindowAggr: windowExpr=[[SUM(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c2, c4] physical_plan -ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] ---BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] -----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=4096 ---------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -----------------CoalesceBatchesExec: target_batch_size=4096 -------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 ---------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true +01)ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +02)--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +03)----SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +04)------CoalesceBatchesExec: target_batch_size=4096 +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 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(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +09)----------------CoalesceBatchesExec: target_batch_size=4096 +10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true # test_window_agg_sort_reversed_plan @@ -1303,18 +1303,18 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c9] +01)Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [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.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] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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) ORDER BY [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.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=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query III SELECT @@ -1344,18 +1344,18 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c9] +01)Projection: aggregate_test_100.c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +04)------WindowAggr: windowExpr=[[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, LAG(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(aggregate_test_100.c9, Int64(2), Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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 }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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 }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as fv1, FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as fv2, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as lag1, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as lag2, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as lead1, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as lead2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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 }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [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(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "FIRST_VALUE(aggregate_test_100.c9) 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 }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query IIIIIII SELECT @@ -1387,19 +1387,19 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c9] +01)Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c9] physical_plan -ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2] ---GlobalLimitExec: skip=0, fetch=5 -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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] -------SortExec: expr=[c9@0 ASC NULLS LAST] ---------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] -----------SortExec: expr=[c9@0 DESC] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] 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] +04)------SortExec: expr=[c9@0 ASC NULLS LAST] +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] +06)----------SortExec: expr=[c9@0 DESC] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query III @@ -1428,22 +1428,22 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.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 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 AS sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[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]] -------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------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]] -----------TableScan: aggregate_test_100 projection=[c1, c2, c9] +01)Projection: aggregate_test_100.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 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 AS sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS rn2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[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]] +04)------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +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 -ProjectionExec: expr=[c9@0 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@3 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@1 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[c9@2 as c9, 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 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, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, 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 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] -------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] ---------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -----------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] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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] ---------------SortExec: expr=[c9@2 DESC,c1@0 DESC] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true +01)ProjectionExec: expr=[c9@0 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@3 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@1 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----ProjectionExec: expr=[c9@2 as c9, 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 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, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, 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 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] +04)------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] +05)--------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +06)----------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] +07)------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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] +08)--------------SortExec: expr=[c9@2 DESC,c1@0 DESC] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true query IIII SELECT @@ -1505,37 +1505,37 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW 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 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 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 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 AS p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS o11 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[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, 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]] -------Projection: null_cases.c1, null_cases.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, 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, 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, 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, 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, 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, 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, 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, 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 ---------WindowAggr: windowExpr=[[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]] -----------WindowAggr: windowExpr=[[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]] -------------WindowAggr: windowExpr=[[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]] ---------------WindowAggr: windowExpr=[[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, 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]] -----------------WindowAggr: windowExpr=[[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, 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]] -------------------WindowAggr: windowExpr=[[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, 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]] ---------------------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]] -----------------------TableScan: null_cases projection=[c1, c2, c3] +01)Projection: SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS a, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS b, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS c, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS d, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS e, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS f, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 11 FOLLOWING AS g, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS i, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW 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 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 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 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 AS p, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS a1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS b1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS c1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS d1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS e1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS f1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND 11 FOLLOWING AS g1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS h1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS j1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS k1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS l1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS m1, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS n1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS o1, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS h11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS j11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS k11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS l11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS LAST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS m11, SUM(null_cases.c1) ORDER BY [null_cases.c3 DESC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS n11, SUM(null_cases.c1) ORDER BY [null_cases.c3 ASC NULLS FIRST] RANGE BETWEEN CURRENT ROW AND UNBOUNDED FOLLOWING AS o11 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[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, 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]] +04)------Projection: null_cases.c1, null_cases.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, 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, 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, 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, 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, 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, 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, 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, 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)--------WindowAggr: windowExpr=[[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)----------WindowAggr: windowExpr=[[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]] +07)------------WindowAggr: windowExpr=[[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]] +08)--------------WindowAggr: windowExpr=[[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, 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]] +09)----------------WindowAggr: windowExpr=[[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, 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)------------------WindowAggr: windowExpr=[[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, 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]] +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 -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] ---GlobalLimitExec: skip=0, fetch=5 -----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] -------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 }] ---------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] -----------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] -------------SortExec: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST] ---------------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] -----------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC] -------------------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] ---------------------SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 DESC] -----------------------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 }] -------------------------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 }] ---------------------------SortExec: expr=[c3@2 DESC NULLS LAST] -----------------------------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 }] -------------------------------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] ---------------------------------SortExec: expr=[c3@2 DESC,c1@0 ASC NULLS LAST] -----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true +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)----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] +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] +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] +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] +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] +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 @@ -1599,19 +1599,19 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.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 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 AS sum2 ---Limit: skip=0, fetch=5 -----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]] -------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]] ---------TableScan: aggregate_test_100 projection=[c1, c9] +01)Projection: aggregate_test_100.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 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 AS sum2 +02)--Limit: skip=0, fetch=5 +03)----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]] +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 -ProjectionExec: expr=[c9@0 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@1 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] ---GlobalLimitExec: skip=0, fetch=5 -----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 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, 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 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] -------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] ---------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] -----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +01)ProjectionExec: expr=[c9@0 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@1 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 +03)----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 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, 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 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] +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)--------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] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1643,19 +1643,19 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.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 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 AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[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]] -------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]] ---------TableScan: aggregate_test_100 projection=[c1, c9] +01)Projection: aggregate_test_100.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 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 AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[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]] +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 -ProjectionExec: expr=[c9@0 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@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@1 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[c9@1 as c9, 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 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, 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 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] -------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] ---------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] -----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +01)ProjectionExec: expr=[c9@0 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@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@1 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----ProjectionExec: expr=[c9@1 as c9, 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 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, 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 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] +04)------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] +05)--------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] +06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1688,25 +1688,25 @@ EXPLAIN SELECT c3, LIMIT 5 ---- logical_plan -Projection: aggregate_test_100.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 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 AS sum2 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS 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 AS 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]] -------Projection: aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c3, aggregate_test_100.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 ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS 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 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]] -----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c2, aggregate_test_100.c3, aggregate_test_100.c9 -------------TableScan: aggregate_test_100 projection=[c2, c3, c4, c9] +01)Projection: aggregate_test_100.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 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 AS sum2 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS 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 AS 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]] +04)------Projection: aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, aggregate_test_100.c3, aggregate_test_100.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 +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3 AS 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 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]] +06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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 -ProjectionExec: expr=[c3@0 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@1 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@2 as sum2] ---GlobalLimitExec: skip=0, fetch=5 -----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 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, 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 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] -------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 }] ---------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] -----------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] -------------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] ---------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -----------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] -------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true +01)ProjectionExec: expr=[c3@0 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@1 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@2 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----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 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, 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 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] +04)------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 }] +05)--------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] +06)----------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] +07)------------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +08)--------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +09)----------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] +10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true query III SELECT c3, @@ -1735,31 +1735,31 @@ EXPLAIN SELECT count(*) as global_count FROM ORDER BY c1 ) AS a ---- logical_plan -Projection: COUNT(*) AS global_count ---Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] -----SubqueryAlias: a -------Projection: ---------Sort: aggregate_test_100.c1 ASC NULLS LAST -----------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] -------------Projection: aggregate_test_100.c1 ---------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") -----------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] +01)Projection: COUNT(*) AS global_count +02)--Aggregate: groupBy=[[]], aggr=[[COUNT(Int64(1)) AS COUNT(*)]] +03)----SubqueryAlias: a +04)------Projection: +05)--------Sort: aggregate_test_100.c1 ASC NULLS LAST +06)----------Aggregate: groupBy=[[aggregate_test_100.c1]], aggr=[[]] +07)------------Projection: aggregate_test_100.c1 +08)--------------Filter: aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434") +09)----------------TableScan: aggregate_test_100 projection=[c1, c13], partial_filters=[aggregate_test_100.c13 != Utf8("C2GT5KVyOPZpgKVl110TyZO0NcJ434")] physical_plan -ProjectionExec: expr=[COUNT(*)@0 as global_count] ---AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 -----------ProjectionExec: expr=[] -------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] ---------------CoalesceBatchesExec: target_batch_size=4096 -----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] ---------------------CoalesceBatchesExec: target_batch_size=4096 -----------------------ProjectionExec: expr=[c1@0 as c1] -------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434 ---------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true +01)ProjectionExec: expr=[COUNT(*)@0 as global_count] +02)--AggregateExec: mode=Final, gby=[], aggr=[COUNT(*)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(*)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=2 +06)----------ProjectionExec: expr=[] +07)------------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[] +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)--------------------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 query I SELECT count(*) as global_count FROM @@ -1786,26 +1786,26 @@ EXPLAIN SELECT c3, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -----Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------Projection: aggregate_test_100.c3, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------TableScan: aggregate_test_100 projection=[c2, c3, c9] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +03)----Projection: aggregate_test_100.c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2 +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------Projection: aggregate_test_100.c3, aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +06)----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------TableScan: aggregate_test_100 projection=[c2, c3, c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 -----ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC] -----------CoalesceBatchesExec: target_batch_size=4096 -------------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 ---------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 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] ---------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortPreservingMergeExec: [c3@0 ASC NULLS LAST], fetch=5 +03)----ProjectionExec: expr=[c3@0 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum2] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c3@0 ASC NULLS LAST,c9@1 DESC] +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +10)------------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 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 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] +11)--------------------SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST] +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true query III @@ -1831,19 +1831,19 @@ query TT EXPLAIN SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 -----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------TableScan: aggregate_test_100 projection=[c1] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +03)----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] ---ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------SortExec: expr=[c1@0 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=4096 -----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c1@0 ASC NULLS LAST] +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query TI SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC @@ -1960,19 +1960,19 @@ query TT EXPLAIN SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 -----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------TableScan: aggregate_test_100 projection=[c1] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS rn1 +03)----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +04)------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] ---ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -------SortExec: expr=[c1@0 ASC NULLS LAST] ---------CoalesceBatchesExec: target_batch_size=4096 -----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true +01)SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c1@0 ASC NULLS LAST] +05)--------CoalesceBatchesExec: target_batch_size=4096 +06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true statement ok set datafusion.optimizer.repartition_sorts = true; @@ -1985,23 +1985,23 @@ EXPLAIN SELECT c1, FROM aggregate_test_100 ORDER BY c1 ASC ---- logical_plan -Sort: aggregate_test_100.c1 ASC NULLS LAST ---Projection: aggregate_test_100.c1, 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 3 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 -----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------WindowAggr: windowExpr=[[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 3 FOLLOWING]] ---------TableScan: aggregate_test_100 projection=[c1, c9] +01)Sort: aggregate_test_100.c1 ASC NULLS LAST +02)--Projection: aggregate_test_100.c1, 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 3 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2 +03)----WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------WindowAggr: windowExpr=[[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 3 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c9] physical_plan -SortExec: expr=[c1@0 ASC NULLS LAST] ---ProjectionExec: expr=[c1@0 as c1, 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 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] -----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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] -------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] ---------SortExec: expr=[c9@1 ASC NULLS LAST] -----------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 3 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 3 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(3)), is_causal: false }], mode=[Sorted] -------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] ---------------CoalesceBatchesExec: target_batch_size=4096 -----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +01)SortExec: expr=[c1@0 ASC NULLS LAST] +02)--ProjectionExec: expr=[c1@0 as c1, 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 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2] +03)----BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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] 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)------SortPreservingMergeExec: [c9@1 ASC NULLS LAST] +05)--------SortExec: expr=[c9@1 ASC NULLS LAST] +06)----------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 3 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 3 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(3)), is_causal: false }], mode=[Sorted] +07)------------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST] +08)--------------CoalesceBatchesExec: target_batch_size=4096 +09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true # test_window_agg_with_global_limit statement ok @@ -2011,20 +2011,20 @@ query TT EXPLAIN SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) ---- logical_plan -Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(aggregate_test_100.c13)]] -----Limit: skip=0, fetch=1 -------Sort: aggregate_test_100.c13 ASC NULLS LAST, fetch=1 ---------TableScan: aggregate_test_100 projection=[c13] +01)Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 +02)--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(aggregate_test_100.c13)]] +03)----Limit: skip=0, fetch=1 +04)------Sort: aggregate_test_100.c13 ASC NULLS LAST, fetch=1 +05)--------TableScan: aggregate_test_100 projection=[c13] physical_plan -ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] -----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------GlobalLimitExec: skip=0, fetch=1 -------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true +01)ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] +02)--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +03)----CoalescePartitionsExec +04)------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------GlobalLimitExec: skip=0, fetch=1 +07)------------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true query ? @@ -2072,26 +2072,26 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 -----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum4 -------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING -----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] -------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] -----------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +03)----Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING AS sum4 +04)------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +05)--------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING +06)----------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING]] +09)----------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c9@0 ASC NULLS LAST] -----ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, 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.c2, 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(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c9@3 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] -----------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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 }] -------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] 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, aggregate_test_100.c2] 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(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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 }] -----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c9@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as sum2, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum3, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as sum4] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, 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.c2, 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(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +05)--------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c9@3 as c9, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] +06)----------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c2, aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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 }] +07)------------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] 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, aggregate_test_100.c2] 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(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +08)--------------WindowAggExec: wdw=[SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.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 }] +09)----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +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 @@ -2125,31 +2125,31 @@ EXPLAIN SELECT c9, LIMIT 5 ---- logical_plan -Projection: t1.c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING 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 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 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 AS sum4 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] -------Projection: t1.c2, t1.c9, t1.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, 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 ---------WindowAggr: windowExpr=[[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]] -----------Projection: t1.c2, t1.c8, t1.c9, t1.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, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING -------------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] ---------------WindowAggr: windowExpr=[[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]] -----------------SubqueryAlias: t1 -------------------Sort: aggregate_test_100.c9 ASC NULLS LAST ---------------------Projection: aggregate_test_100.c1, aggregate_test_100.c2, aggregate_test_100.c8, aggregate_test_100.c9, aggregate_test_100.c1 AS c1_alias -----------------------TableScan: aggregate_test_100 projection=[c1, c2, c8, c9] +01)Projection: t1.c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING 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 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 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 AS sum4 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +04)------Projection: t1.c2, t1.c9, t1.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, 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 +05)--------WindowAggr: windowExpr=[[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]] +06)----------Projection: t1.c2, t1.c8, t1.c9, t1.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, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING +07)------------WindowAggr: windowExpr=[[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[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]] +09)----------------SubqueryAlias: t1 +10)------------------Sort: aggregate_test_100.c9 ASC NULLS LAST +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 -ProjectionExec: expr=[c9@0 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@2 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@4 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@1 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@3 as sum4] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[c9@1 as c9, 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 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@4 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@5 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, 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 SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] -------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] ---------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] -----------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 }] -------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] ---------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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] -----------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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 }] -------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] ---------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +01)ProjectionExec: expr=[c9@0 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@2 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@4 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@1 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@3 as sum4] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----ProjectionExec: expr=[c9@1 as c9, 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 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@4 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@5 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, 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 SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +04)------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] +05)--------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] +06)----------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 }] +07)------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +08)--------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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] +09)----------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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 }] +10)------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +11)--------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +12)----------------------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, @@ -2176,24 +2176,24 @@ EXPLAIN SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FO FROM aggregate_test_100 ORDER BY c9 LIMIT 5 ---- logical_plan -Projection: sum1, sum2 ---Limit: skip=0, fetch=5 -----Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 -------Projection: 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 AS sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING AS sum2, aggregate_test_100.c9 ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING]] -----------Projection: aggregate_test_100.c1, aggregate_test_100.c9, aggregate_test_100.c12, 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 -------------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]] ---------------TableScan: aggregate_test_100 projection=[c1, c2, c9, c12] +01)Projection: sum1, sum2 +02)--Limit: skip=0, fetch=5 +03)----Sort: aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +04)------Projection: 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 AS sum1, SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING AS sum2, aggregate_test_100.c9 +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST] GROUPS BETWEEN 5 PRECEDING AND 3 PRECEDING]] +06)----------Projection: aggregate_test_100.c1, aggregate_test_100.c9, aggregate_test_100.c12, 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 +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 -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] -----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] -------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] ---------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] -----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, 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@4 as 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] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true +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] +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] +06)----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, 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@4 as 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] +07)------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Groups, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9, c12], has_header=true query RR SELECT SUM(c12) OVER(ORDER BY c1, c2 GROUPS BETWEEN 1 PRECEDING AND 1 FOLLOWING) as sum1, @@ -2218,18 +2218,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 ASC NULLS LAST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 ASC NULLS LAST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 ASC NULLS LAST] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 ASC NULLS LAST] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2257,18 +2257,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2296,19 +2296,19 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 DESC NULLS FIRST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 DESC NULLS FIRST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[rn1@1 DESC] -----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[rn1@1 DESC] +03)----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +04)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2339,19 +2339,19 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] -----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +04)------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, @@ -2393,18 +2393,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true # This test shows that ordering equivalence can keep track of complex expressions (not just Column expressions) # during ordering satisfy analysis. In the final plan we should only see single SortExec. @@ -2417,18 +2417,18 @@ EXPLAIN SELECT c5, c9, rn1 FROM (SELECT c5, c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST, fetch=5 -----Sort: CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST -------Projection: aggregate_test_100.c5, aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c5, c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST, fetch=5 +03)----Sort: CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c5, aggregate_test_100.c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [CAST(aggregate_test_100.c9 AS Int32) + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c5, c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c5@0 as c5, c9@1 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[CAST(c9@1 AS Int32) + c5@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c5@0 as c5, c9@1 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 + aggregate_test_100.c5 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[CAST(c9@1 AS Int32) + c5@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c5, c9], has_header=true # Ordering equivalence should be preserved during cast expression query TT @@ -2440,18 +2440,18 @@ EXPLAIN SELECT c9, rn1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: rn1 ASC NULLS LAST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS Int64) AS rn1 ---------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: rn1 ASC NULLS LAST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS Int64) AS rn1 +05)--------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 AS Int64) as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c9@0 as c9, CAST(ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 AS Int64) as rn1] +03)----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +04)------SortExec: expr=[c9@0 DESC] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true # The following query has type error. We should test the error could be detected # from either the logical plan (when `skip_failed_rules` is set to `false`) or @@ -2542,27 +2542,27 @@ EXPLAIN SELECT LIMIT 5; ---- logical_plan -Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, sumr1, sumr2, sumr3, minr1, minr2, minr3, maxr1, maxr2, maxr3, cntr1, cntr2, sum4, cnt3 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.inc_col DESC NULLS FIRST, fetch=5 -------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING AS sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING AS sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt3, annotated_data_finite.inc_col ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] -----------Projection: CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, annotated_data_finite.inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING -------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] -----------------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 -------------------TableScan: annotated_data_finite projection=[ts, inc_col, desc_col] +01)Projection: sum1, sum2, sum3, min1, min2, min3, max1, max2, max3, cnt1, cnt2, sumr1, sumr2, sumr3, minr1, minr2, minr3, maxr1, maxr2, maxr3, cntr1, cntr2, sum4, cnt3 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_finite.inc_col DESC NULLS FIRST, fetch=5 +04)------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS sum2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS sum3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS min1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS min2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS min3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS max1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS max2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS max3, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS cnt1, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt2, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING AS sumr1, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING AS sumr2, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING AS sumr3, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS minr1, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS minr2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS minr3, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS maxr1, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING AS maxr2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING AS maxr3, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS cntr1, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cntr2, SUM(annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS sum4, COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS cnt3, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, COUNT(Int64(1)) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +06)----------Projection: CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, annotated_data_finite.inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING +07)------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col AS annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(Int64(1)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING AS COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING]] +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 -GlobalLimitExec: skip=0, fetch=5 ---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] -----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] -------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] ---------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] -----------ProjectionExec: expr=[CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col@0 as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, inc_col@3 as inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] -------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] 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] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 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(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] 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(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] -----------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +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] +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] +06)----------ProjectionExec: expr=[CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col@0 as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, inc_col@3 as inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] 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] +08)--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(8)), end_bound: Following(Int32(1)), is_causal: false }, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 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(1)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(5)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(2)), end_bound: Following(Int32(6)), is_causal: false }, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] 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(1)), end_bound: Following(UInt64(8)), is_causal: false }], mode=[Sorted] +09)----------------ProjectionExec: expr=[CAST(desc_col@2 AS Int64) as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col, desc_col@2 as desc_col] +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col, desc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2635,19 +2635,19 @@ EXPLAIN SELECT LIMIT 5; ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 -----Projection: annotated_data_finite.ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 -------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] ---------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] -----------TableScan: annotated_data_finite projection=[ts, inc_col] +01)Limit: skip=0, fetch=5 +02)--Sort: annotated_data_finite.ts DESC NULLS FIRST, fetch=5 +03)----Projection: annotated_data_finite.ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING AS leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING AS leadr2 +04)------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, NTH_VALUE(annotated_data_finite.inc_col, Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +05)--------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LAG(annotated_data_finite.inc_col, Int64(2), Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(-1), Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING, LEAD(annotated_data_finite.inc_col, Int64(4), Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING]] +06)----------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[ts@0 DESC] -----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] -------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[ts@0 DESC] +03)----ProjectionExec: expr=[ts@0 as ts, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2] +04)------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "NTH_VALUE(annotated_data_finite.inc_col,Int64(5)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "DENSE_RANK() ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 10 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(10)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LAG(annotated_data_finite.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)), is_causal: false }, LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "LEAD(annotated_data_finite.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 10 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(1)), end_bound: Following(UInt64(10)), is_causal: false }], mode=[Sorted] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -2706,23 +2706,23 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS avg2, annotated_data_finite.inc_col ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_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 ---------------TableScan: annotated_data_finite projection=[ts, inc_col] +01)Projection: sum1, sum2, min1, min2, max1, max2, count1, count2, avg1, avg2 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 +04)------Projection: SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS sum1, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS min1, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS min2, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS max1, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS max2, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS count1, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING AS avg1, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS avg2, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, AVG(CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col AS annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_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 +08)--------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---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] -----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] -------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] ---------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] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] -------------ProjectionExec: expr=[CAST(inc_col@1 AS Float64) as CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +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] +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] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] +07)------------ProjectionExec: expr=[CAST(inc_col@1 AS Float64) as CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_col, CAST(inc_col@1 AS Int64) as CAST(annotated_data_finite.inc_col AS Int64)annotated_data_finite.inc_col, ts@0 as ts, inc_col@1 as inc_col] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIIIIIIRR SELECT @@ -2761,21 +2761,21 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 -------Projection: FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING 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 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 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 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 AS nth_value1, annotated_data_finite.inc_col ---------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, 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]] -----------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]] -------------TableScan: annotated_data_finite projection=[ts, inc_col] +01)Projection: first_value1, first_value2, last_value1, last_value2, nth_value1 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_finite.inc_col ASC NULLS LAST, fetch=5 +04)------Projection: FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING 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 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 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 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 AS nth_value1, annotated_data_finite.inc_col +05)--------WindowAggr: windowExpr=[[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, 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]] +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 -GlobalLimitExec: skip=0, fetch=5 ---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] -----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] -------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] ---------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] -----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true +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] +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] +06)----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[ts, inc_col], output_ordering=[ts@0 ASC NULLS LAST], has_header=true query IIIII SELECT @@ -2807,21 +2807,21 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: sum1, sum2, count1, count2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 -------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col ---------------TableScan: annotated_data_infinite projection=[ts, inc_col] +01)Projection: sum1, sum2, count1, count2 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 +04)------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +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 -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] -----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), 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] -------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] ---------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +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 @@ -2853,21 +2853,21 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: sum1, sum2, count1, count2 ---Limit: skip=0, fetch=5 -----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 -------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] -------------Projection: CAST(annotated_data_infinite.inc_col AS Int64) AS CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, annotated_data_infinite.ts, annotated_data_infinite.inc_col ---------------TableScan: annotated_data_infinite projection=[ts, inc_col] +01)Projection: sum1, sum2, count1, count2 +02)--Limit: skip=0, fetch=5 +03)----Sort: annotated_data_infinite.ts ASC NULLS LAST, fetch=5 +04)------Projection: SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING AS count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING AS count2, annotated_data_infinite.ts +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col AS annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING]] +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 -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as sum1, SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as sum2, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as count1, COUNT(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as count2] -----BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)), 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] -------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] ---------ProjectionExec: expr=[CAST(inc_col@1 AS Int64) as CAST(annotated_data_infinite.inc_col AS Int64)annotated_data_infinite.inc_col, ts@0 as ts, inc_col@1 as inc_col] -----------StreamingTableExec: partition_sizes=1, projection=[ts, inc_col], infinite_source=true, output_ordering=[ts@0 ASC NULLS LAST] +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 @@ -2950,28 +2950,28 @@ EXPLAIN SELECT a, b, c, LIMIT 5 ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.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 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 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 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 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 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 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 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 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 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 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 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 AS sum12 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] -------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] ---------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] -----------------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 -------------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +01)Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.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 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 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 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 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 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 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 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 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 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 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 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 AS sum12 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] +07)------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] +08)--------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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, SUM(CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c AS 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]] +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 -ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 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@7 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@8 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@13 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@14 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@3 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@4 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@9 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@10 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@5 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@6 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@11 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@12 as sum12] ---GlobalLimitExec: skip=0, fetch=5 -----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.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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] -------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] ---------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])] -----------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] -------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] ---------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] -----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.a, annotated_data_infinite2.b] 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.a, annotated_data_infinite2.b] 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] -------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] ---------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 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@7 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@8 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@13 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@14 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@3 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@4 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@9 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@10 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@5 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@6 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@11 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@12 as sum12] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----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.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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] +04)------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] +05)--------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])] +06)----------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] +07)------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] +08)--------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] +09)----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.a, annotated_data_infinite2.b] 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.a, annotated_data_infinite2.b] 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] +10)------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +11)--------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3019,34 +3019,34 @@ EXPLAIN SELECT a, b, c, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: annotated_data_finite2.c ASC NULLS LAST, fetch=5 -----Projection: annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] ---------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] -----------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] -------------------Projection: CAST(annotated_data_finite2.c AS Int64) AS CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, annotated_data_finite2.d ---------------------TableScan: annotated_data_finite2 projection=[a, b, c, d] +01)Limit: skip=0, fetch=5 +02)--Sort: annotated_data_finite2.c ASC NULLS LAST, fetch=5 +03)----Projection: annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING AS sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING AS sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING AS sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW AS sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING AS sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING AS sum12 +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +07)------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING]] +08)--------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW]] +09)----------------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING, SUM(CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c AS annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING]] +10)------------------Projection: CAST(annotated_data_finite2.c AS Int64) AS CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.c, annotated_data_finite2.d +11)--------------------TableScan: annotated_data_finite2 projection=[a, b, c, d] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST] -----ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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=[Sorted] ---------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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=[Sorted] -------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] ---------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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] -----------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST] -------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] ---------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] -----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] -------------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] ---------------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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] -----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c@2 ASC NULLS LAST] +03)----ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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=[Sorted] +05)--------SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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=[Sorted] +07)------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] +08)--------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.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] +09)----------------SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST] +10)------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[Sorted] +11)--------------------SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST] +12)----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] +13)------------------------SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST] +14)--------------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.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] +15)----------------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query IIIIIIIIIIIIIII SELECT a, b, c, @@ -3098,20 +3098,20 @@ EXPLAIN SELECT * FROM (SELECT *, ROW_NUMBER() OVER(ORDER BY a ASC) as rn1 ORDER BY rn1 ASC ---- logical_plan -Sort: rn1 ASC NULLS LAST ---Filter: rn1 < UInt64(50) -----Limit: skip=0, fetch=5 -------Sort: rn1 ASC NULLS LAST, fetch=5 ---------Projection: annotated_data_infinite2.a0, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 -----------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] +01)Sort: rn1 ASC NULLS LAST +02)--Filter: rn1 < UInt64(50) +03)----Limit: skip=0, fetch=5 +04)------Sort: rn1 ASC NULLS LAST, fetch=5 +05)--------Projection: annotated_data_infinite2.a0, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS rn1 +06)----------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------TableScan: annotated_data_infinite2 projection=[a0, a, b, c, d] physical_plan -CoalesceBatchesExec: target_batch_size=4096 ---FilterExec: rn1@5 < 50 -----GlobalLimitExec: skip=0, fetch=5 -------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] ---------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] +01)CoalesceBatchesExec: target_batch_size=4096 +02)--FilterExec: rn1@5 < 50 +03)----GlobalLimitExec: skip=0, fetch=5 +04)------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as rn1] +05)--------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +06)----------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] # this is a negative test for asserting that window functions (other than ROW_NUMBER) # are not added to ordering equivalence @@ -3125,19 +3125,19 @@ EXPLAIN SELECT c9, sum1 FROM (SELECT c9, LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: sum1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 -----Sort: aggregate_test_100.c9 DESC NULLS FIRST -------Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 ---------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------TableScan: aggregate_test_100 projection=[c9] +01)Limit: skip=0, fetch=5 +02)--Sort: sum1 ASC NULLS LAST, aggregate_test_100.c9 DESC NULLS FIRST, fetch=5 +03)----Sort: aggregate_test_100.c9 DESC NULLS FIRST +04)------Projection: aggregate_test_100.c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 +05)--------WindowAggr: windowExpr=[[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------TableScan: aggregate_test_100 projection=[c9] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] -----ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] -------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------SortExec: expr=[c9@0 DESC] -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[sum1@1 ASC NULLS LAST,c9@0 DESC] +03)----ProjectionExec: expr=[c9@0 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1] +04)------BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 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(UInt64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------SortExec: expr=[c9@0 DESC] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true # Query below should work when its input is unbounded # because ordering of ROW_NUMBER, RANK result is added to the ordering equivalence @@ -3210,23 +3210,23 @@ SUM(a) OVER(partition by b, a order by c) as sum2, FROM annotated_data_infinite2; ---- logical_plan -Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 ---WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d ---------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +01)Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 +02)--WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +08)--------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] ---BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] -----ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] ---------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] +02)--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] +03)----ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +08)--------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] statement ok set datafusion.execution.target_partitions = 2; @@ -3241,32 +3241,32 @@ EXPLAIN SELECT SUM(a) OVER(partition by a, b order by c) as sum1, FROM annotated_data_infinite2; ---- logical_plan -Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 ---WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d ---------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] +01)Projection: SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum4 +02)--WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +06)----------WindowAggr: windowExpr=[[SUM(CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a AS annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +07)------------Projection: CAST(annotated_data_infinite2.a AS Int64) AS CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, annotated_data_infinite2.a, annotated_data_infinite2.b, annotated_data_infinite2.c, annotated_data_infinite2.d +08)--------------TableScan: annotated_data_infinite2 projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] ---BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] -----CoalesceBatchesExec: target_batch_size=4096 -------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST,a@1 ASC NULLS LAST ---------ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------CoalesceBatchesExec: target_batch_size=4096 ---------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] -------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -------------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST -----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ---------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +01)ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as sum4] +02)--BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------RepartitionExec: partitioning=Hash([d@2], 2), input_partitions=2, preserve_order=true, sort_exprs=CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST,a@1 ASC NULLS LAST +05)--------ProjectionExec: expr=[CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@1 as a, d@4 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@7 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +07)------------CoalesceBatchesExec: target_batch_size=4096 +08)--------------RepartitionExec: partitioning=Hash([b@2, a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST +09)----------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[PartiallySorted([0])] +10)------------------CoalesceBatchesExec: target_batch_size=4096 +11)--------------------RepartitionExec: partitioning=Hash([a@1, d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST +12)----------------------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +13)------------------------CoalesceBatchesExec: target_batch_size=4096 +14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST,CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a@0 ASC NULLS LAST +15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as CAST(annotated_data_infinite2.a AS Int64)annotated_data_infinite2.a, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] # reset the partition number 1 again statement ok @@ -3311,23 +3311,23 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 -------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING -----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -------------TableScan: aggregate_test_100 projection=[c3, c11, c12] +01)Limit: skip=0, fetch=5 +02)--Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +03)----Projection: aggregate_test_100.c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS max1 +04)------WindowAggr: windowExpr=[[MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------Projection: aggregate_test_100.c3, aggregate_test_100.c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +06)----------WindowAggr: windowExpr=[[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +07)------------TableScan: aggregate_test_100 projection=[c3, c11, c12] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] -----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] -------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 }], mode=[Sorted] ---------SortExec: expr=[c12@1 ASC NULLS LAST] -----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] -------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, 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 }] ---------------SortExec: expr=[c11@1 ASC NULLS LAST] -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--SortExec: TopK(fetch=5), expr=[c3@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[c3@0 as c3, MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@2 as max1] +04)------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 }], mode=[Sorted] +05)--------SortExec: expr=[c12@1 ASC NULLS LAST] +06)----------ProjectionExec: expr=[c3@0 as c3, c12@2 as c12, MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@3 as MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +07)------------WindowAggExec: wdw=[MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(aggregate_test_100.c12) PARTITION BY [aggregate_test_100.c11] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: Float64, nullable: true, 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 }] +08)--------------SortExec: expr=[c11@1 ASC NULLS LAST] +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c11, c12], has_header=true # window1 spec is used multiple times under different aggregations. # The query should still work. @@ -3357,20 +3357,20 @@ EXPLAIN SELECT LIMIT 5 ---- logical_plan -Projection: min1, max1 ---Limit: skip=0, fetch=5 -----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 -------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1, aggregate_test_100.c3 ---------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]] -----------TableScan: aggregate_test_100 projection=[c3, c12] +01)Projection: min1, max1 +02)--Limit: skip=0, fetch=5 +03)----Sort: aggregate_test_100.c3 ASC NULLS LAST, fetch=5 +04)------Projection: MAX(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MIN(aggregate_test_100.c12) ORDER BY [aggregate_test_100.c12 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1, aggregate_test_100.c3 +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 -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] -----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] -------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] ---------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] -----------SortExec: expr=[c12@1 ASC NULLS LAST] -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true +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] +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] +06)----------SortExec: expr=[c12@1 ASC NULLS LAST] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c12], has_header=true # window2 spec is not defined statement error DataFusion error: Error during planning: The window window2 is not defined! @@ -3416,14 +3416,14 @@ FROM multiple_ordered_table where b=0 ---- logical_plan -WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---Filter: multiple_ordered_table.b = Int32(0) -----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +01)WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +02)--Filter: multiple_ordered_table.b = Int32(0) +03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan -BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---CoalesceBatchesExec: target_batch_size=4096 -----FilterExec: b@2 = 0 -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +01)BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +02)--CoalesceBatchesExec: target_batch_size=4096 +03)----FilterExec: b@2 = 0 +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true # Since column b is constant after filter b=0, # window requirement b ASC, d ASC can be satisfied @@ -3434,15 +3434,15 @@ FROM multiple_ordered_table where b=0 ---- logical_plan -WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---Filter: multiple_ordered_table.b = Int32(0) -----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] +01)WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.a AS Int64)) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +02)--Filter: multiple_ordered_table.b = Int32(0) +03)----TableScan: multiple_ordered_table projection=[a0, a, b, c, d], partial_filters=[multiple_ordered_table.b = Int32(0)] physical_plan -BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---SortExec: expr=[d@4 ASC NULLS LAST] -----CoalesceBatchesExec: target_batch_size=4096 -------FilterExec: b@2 = 0 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true +01)BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.b ASC NULLS LAST, multiple_ordered_table.d 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +02)--SortExec: expr=[d@4 ASC NULLS LAST] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------FilterExec: b@2 = 0 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]], has_header=true # Create an unbounded source where there is multiple orderings. @@ -3468,17 +3468,17 @@ EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, FROM multiple_ordered_table ---- logical_plan -Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 ---WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW -------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: multiple_ordered_table projection=[a, b, c, d] +01)Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 +02)--WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +04)------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] ---BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] -------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] +02)--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +04)------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query TT EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c @@ -3488,42 +3488,42 @@ FROM( WHERE d=0) ---- logical_plan -Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c ---WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Filter: multiple_ordered_table.d = Int32(0) -------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] +01)Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c +02)--WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Filter: multiple_ordered_table.d = Int32(0) +04)------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] physical_plan -ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] ---BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CoalesceBatchesExec: target_batch_size=4096 -------FilterExec: d@1 = 0 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] +02)--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------FilterExec: d@1 = 0 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) FROM multiple_ordered_table; ---- logical_plan -Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----TableScan: multiple_ordered_table projection=[a, c, d] +01)Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) FROM multiple_ordered_table; ---- logical_plan -Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----TableScan: multiple_ordered_table projection=[a, b, c, d] +01)Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan -ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b 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(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true query I SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) @@ -3557,16 +3557,16 @@ EXPLAIN SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 LIMIT 5 ---- logical_plan -Limit: skip=0, fetch=5 ---Sort: multiple_ordered_table.c ASC NULLS LAST, fetch=5 -----Projection: multiple_ordered_table.c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS nv1 -------WindowAggr: windowExpr=[[NTH_VALUE(multiple_ordered_table.c, Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] ---------TableScan: multiple_ordered_table projection=[c] +01)Limit: skip=0, fetch=5 +02)--Sort: multiple_ordered_table.c ASC NULLS LAST, fetch=5 +03)----Projection: multiple_ordered_table.c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS nv1 +04)------WindowAggr: windowExpr=[[NTH_VALUE(multiple_ordered_table.c, Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +05)--------TableScan: multiple_ordered_table projection=[c] physical_plan -GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c@0 as c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] -----WindowAggExec: wdw=[NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] -------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true +01)GlobalLimitExec: skip=0, fetch=5 +02)--ProjectionExec: expr=[c@0 as c, NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as nv1] +03)----WindowAggExec: wdw=[NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "NTH_VALUE(multiple_ordered_table.c,Int64(2)) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int32(NULL)), is_causal: false }] +04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true query II SELECT c, NTH_VALUE(c, 2) OVER(order by c DESC) as nv1 @@ -3609,18 +3609,18 @@ WINDOW sliding_window AS ( ORDER BY c ---- logical_plan -Sort: multiple_ordered_table_inf.c ASC NULLS LAST ---Projection: multiple_ordered_table_inf.a0, multiple_ordered_table_inf.a, multiple_ordered_table_inf.b, multiple_ordered_table_inf.c, multiple_ordered_table_inf.d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW AS avg_d -----WindowAggr: windowExpr=[[AVG(CAST(multiple_ordered_table_inf.d AS Float64)) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW]] -------TableScan: multiple_ordered_table_inf projection=[a0, a, b, c, d] +01)Sort: multiple_ordered_table_inf.c ASC NULLS LAST +02)--Projection: multiple_ordered_table_inf.a0, multiple_ordered_table_inf.a, multiple_ordered_table_inf.b, multiple_ordered_table_inf.c, multiple_ordered_table_inf.d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW AS avg_d +03)----WindowAggr: windowExpr=[[AVG(CAST(multiple_ordered_table_inf.d AS Float64)) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW]] +04)------TableScan: multiple_ordered_table_inf projection=[a0, a, b, c, d] physical_plan -SortPreservingMergeExec: [c@3 ASC NULLS LAST] ---ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] -----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] -------CoalesceBatchesExec: target_batch_size=4096 ---------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST -----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] +01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] +02)--ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW@5 as avg_d] +03)----BoundedWindowAggExec: wdw=[AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "AVG(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] +04)------CoalesceBatchesExec: target_batch_size=4096 +05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] # CTAS with NTILE function statement ok @@ -3945,14 +3945,14 @@ query TT EXPLAIN SELECT *, SUM(amount) OVER (ORDER BY sn) as sum1 FROM table_with_pk; ---- logical_plan -Projection: table_with_pk.sn, table_with_pk.ts, table_with_pk.currency, table_with_pk.amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 ---WindowAggr: windowExpr=[[SUM(CAST(table_with_pk.amount AS Float64)) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----TableScan: table_with_pk projection=[sn, ts, currency, amount] +01)Projection: table_with_pk.sn, table_with_pk.ts, table_with_pk.currency, table_with_pk.amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS sum1 +02)--WindowAggr: windowExpr=[[SUM(CAST(table_with_pk.amount AS Float64)) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----TableScan: table_with_pk projection=[sn, ts, currency, amount] physical_plan -ProjectionExec: expr=[sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1] ---BoundedWindowAggExec: wdw=[SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] -----SortExec: expr=[sn@0 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[sn@0 as sn, ts@1 as ts, currency@2 as currency, amount@3 as amount, SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum1] +02)--BoundedWindowAggExec: wdw=[SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(table_with_pk.amount) ORDER BY [table_with_pk.sn ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow, is_causal: true }], mode=[Sorted] +03)----SortExec: expr=[sn@0 ASC NULLS LAST] +04)------MemoryExec: partitions=1, partition_sizes=[1] # test ROW_NUMBER window function returns correct data_type query T @@ -4060,21 +4060,21 @@ explain SELECT c3, limit 5 ---- logical_plan -Projection: aggregate_test_100.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 AS sum2, sum1 ---Limit: skip=0, fetch=5 -----WindowAggr: windowExpr=[[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]] -------Projection: aggregate_test_100.c3, aggregate_test_100.c4, aggregate_test_100.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 AS sum1 ---------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]] -----------TableScan: aggregate_test_100 projection=[c3, c4, c9] +01)Projection: aggregate_test_100.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 AS sum2, sum1 +02)--Limit: skip=0, fetch=5 +03)----WindowAggr: windowExpr=[[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]] +04)------Projection: aggregate_test_100.c3, aggregate_test_100.c4, aggregate_test_100.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 AS sum1 +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 -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@2 as sum2, sum1@1 as sum1] ---GlobalLimitExec: skip=0, fetch=5 -----ProjectionExec: expr=[c3@0 as c3, sum1@3 as sum1, 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 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] -------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 }] ---------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] -----------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] -------------SortExec: expr=[c3@0 + c4@1 DESC] ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true +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@2 as sum2, sum1@1 as sum1] +02)--GlobalLimitExec: skip=0, fetch=5 +03)----ProjectionExec: expr=[c3@0 as c3, sum1@3 as sum1, 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 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] +04)------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 }] +05)--------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] +06)----------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] +07)------------SortExec: expr=[c3@0 + c4@1 DESC] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true query III SELECT c3, @@ -4105,19 +4105,19 @@ query TT EXPLAIN select count(*) over (partition by a order by a) from (select * from a where a = 1); ---- logical_plan -Projection: COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW ---WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] -----Filter: a.a = Int64(1) -------TableScan: a projection=[a] +01)Projection: COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +02)--WindowAggr: windowExpr=[[COUNT(Int64(1)) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +03)----Filter: a.a = Int64(1) +04)------TableScan: a projection=[a] physical_plan -ProjectionExec: expr=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ---BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a 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] -----CoalesceBatchesExec: target_batch_size=4096 -------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------CoalesceBatchesExec: target_batch_size=4096 -------------FilterExec: a@0 = 1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +02)--BoundedWindowAggExec: wdw=[COUNT(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "COUNT(*) PARTITION BY [a.a] ORDER BY [a.a 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] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------FilterExec: a@0 = 1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] query I select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); @@ -4128,19 +4128,19 @@ query TT EXPLAIN select ROW_NUMBER() over (partition by a) from (select * from a where a = 1); ---- logical_plan -Projection: ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING ---WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] -----Filter: a.a = Int64(1) -------TableScan: a projection=[a] +01)Projection: ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING +02)--WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] +03)----Filter: a.a = Int64(1) +04)------TableScan: a projection=[a] physical_plan -ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] ---BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] -----CoalesceBatchesExec: target_batch_size=4096 -------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 ---------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -----------CoalesceBatchesExec: target_batch_size=4096 -------------FilterExec: a@0 = 1 ---------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING] +02)--BoundedWindowAggExec: wdw=[ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] +03)----CoalesceBatchesExec: target_batch_size=4096 +04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------CoalesceBatchesExec: target_batch_size=4096 +07)------------FilterExec: a@0 = 1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] # LAG window function IGNORE/RESPECT NULLS support with ascending order and default offset 1 query TTTTTT From 4d02e9e968fcd0c82720609dfa81627c0dd6917d Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 17 Apr 2024 17:20:28 +0300 Subject: [PATCH 56/85] fixing tpch queries --- .../optimize_projections.rs | 59 ++++++++++++++++++- 1 file changed, 58 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index be4058ce277e..6ff11ae78487 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3117,7 +3117,7 @@ impl ProjectionOptimizer { .map(|child| child.plan.clone()) .collect(), )?; - update_mapping(&mut self, all_mappings) + update_mapping_cross(&mut self, all_mappings) } else if let Some(projection) = plan_any.downcast_ref::() { self.plan = rewrite_projection( projection, @@ -4762,6 +4762,63 @@ fn update_mapping( } } +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.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: HashMap, left_size: usize, From 23b306eae539c542f0362eaa0be95941d55d868f Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 17 Apr 2024 17:27:22 +0300 Subject: [PATCH 57/85] Update physical_planner.rs --- datafusion/core/src/physical_planner.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 657974a18c43..fd05770dc1ca 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -89,8 +89,8 @@ use datafusion_expr::expr::{ use datafusion_expr::expr_rewriter::unnormalize_cols; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{ - expr_vec_fmt, DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, - StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, + expr_vec_fmt, DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, + ScalarFunctionDefinition, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; From 0bc242150a7ffaadaf207ab9b7df4402e35531e0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 18 Apr 2024 12:42:29 +0300 Subject: [PATCH 58/85] Update optimize_projections.rs --- .../optimize_projections.rs | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 6ff11ae78487..d5978a663e95 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -201,6 +201,30 @@ impl ProjectionOptimizer { }; } + if let Some(limit) = projection_input.as_any().downcast_ref::() { + let clone = self.plan.clone(); + let projection = clone.as_any().downcast_ref::().unwrap(); + // 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() { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .flat_map(|e| collect_columns(&projection.expr()[e.index()].0)) + .collect::>(); + return Ok(self); + } + + let new_p = self.plan.with_new_children(limit.children())?; + self.plan = Arc::new(GlobalLimitExec::new( + new_p.clone(), + limit.skip(), + limit.fetch(), + )); + self.children_nodes[0].plan = new_p; + self.children_nodes[0].required_columns = self.required_columns.clone(); + return Ok(self); + } + // Source providers: if projection_input.as_any().is::() { return self.try_projected_csv(); From 09bfda255ce752bf5b3810bf45e7db125efd89d9 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 18 Apr 2024 12:43:41 +0300 Subject: [PATCH 59/85] Remove unnecessary projections --- datafusion/sqllogictest/test_files/window.slt | 123 +++++++++--------- 1 file changed, 58 insertions(+), 65 deletions(-) diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 59b816c317c0..438f0526b56d 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1435,15 +1435,14 @@ 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@0 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@3 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@1 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn2] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----ProjectionExec: expr=[c9@2 as c9, 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 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, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, 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 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] -04)------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] -05)--------SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST] -06)----------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] -07)------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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] -08)--------------SortExec: expr=[c9@2 DESC,c1@0 DESC] -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true +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] +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] +06)----------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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] +07)------------SortExec: expr=[c9@2 DESC,c1@0 DESC] +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c9], has_header=true query IIII SELECT @@ -1605,13 +1604,12 @@ 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@0 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@1 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 -03)----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 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, 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 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] -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)--------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] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +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] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1649,13 +1647,12 @@ 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@0 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@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@1 as sum2] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----ProjectionExec: expr=[c9@1 as c9, 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 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, 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 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] -04)------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] -05)--------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] -06)----------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] -07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true +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] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true query III SELECT @@ -1696,17 +1693,16 @@ logical_plan 06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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@0 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@1 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@2 as sum2] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----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 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, 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 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] -04)------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 }] -05)--------ProjectionExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] -06)----------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] -07)------------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -08)--------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] -09)----------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true +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.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] +06)----------SortPreservingMergeExec: [aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +07)------------SortExec: expr=[aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3@0 DESC,c9@3 DESC,c2@1 ASC NULLS LAST] +08)--------------ProjectionExec: expr=[c3@1 + c4@2 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_test_100.c3, c2@0 as c2, c3@1 as c3, c9@3 as c9] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true query III SELECT c3, @@ -2138,18 +2134,17 @@ 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@0 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@2 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@4 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@1 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@3 as sum4] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----ProjectionExec: expr=[c9@1 as c9, 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 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@4 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@5 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, 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 SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] -04)------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] -05)--------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] -06)----------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 }] -07)------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] -08)--------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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] -09)----------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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 }] -10)------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] -11)--------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true +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 }] +06)----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 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@5 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@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] 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] +08)--------------WindowAggExec: wdw=[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: Ok(Field { name: "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", 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 }] +09)----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +10)------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +11)--------------------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, @@ -2961,17 +2956,16 @@ 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@0 as a, b@1 as b, c@2 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@7 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@8 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@13 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@14 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@3 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@4 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@9 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@10 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@5 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@6 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@11 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@12 as sum12] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----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.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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, 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 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] -04)------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] -05)--------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])] -06)----------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] -07)------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] -08)--------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] -09)----------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.a, annotated_data_infinite2.b] 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.a, annotated_data_infinite2.b] 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] -10)------------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -11)--------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] +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] +06)----------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)), is_causal: false }], mode=[PartiallySorted([0])] +07)------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.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: Ok(Field { name: "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", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow, is_causal: true }], mode=[PartiallySorted([0, 1])] +08)--------------BoundedWindowAggExec: wdw=[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: Ok(Field { name: "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", 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.a, annotated_data_infinite2.b] 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.a, annotated_data_infinite2.b] 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] +09)----------------ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_infinite2.c AS Int64)annotated_data_infinite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d] +10)------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query IIIIIIIIIIIIIII SELECT a, b, c, @@ -4067,14 +4061,13 @@ 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@2 as sum2, sum1@1 as sum1] -02)--GlobalLimitExec: skip=0, fetch=5 -03)----ProjectionExec: expr=[c3@0 as c3, sum1@3 as sum1, 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 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] -04)------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 }] -05)--------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] -06)----------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] -07)------------SortExec: expr=[c3@0 + c4@1 DESC] -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true +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] +06)----------SortExec: expr=[c3@0 + c4@1 DESC] +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3, c4, c9], has_header=true query III SELECT c3, From cef1c3634d03656cc45d281bd30bd850b3f7f4f5 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Thu, 18 Apr 2024 13:32:26 +0300 Subject: [PATCH 60/85] Update optimize_projections.rs --- .../optimize_projections.rs | 63 ++++++++++++------- 1 file changed, 41 insertions(+), 22 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index d5978a663e95..9a42f9ab7e0e 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -201,28 +201,21 @@ impl ProjectionOptimizer { }; } - if let Some(limit) = projection_input.as_any().downcast_ref::() { - let clone = self.plan.clone(); - let projection = clone.as_any().downcast_ref::().unwrap(); - // 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() { - self.children_nodes[0].required_columns = self - .required_columns - .iter() - .flat_map(|e| collect_columns(&projection.expr()[e.index()].0)) - .collect::>(); - return Ok(self); - } - - let new_p = self.plan.with_new_children(limit.children())?; - self.plan = Arc::new(GlobalLimitExec::new( - new_p.clone(), - limit.skip(), - limit.fetch(), - )); - self.children_nodes[0].plan = new_p; - self.children_nodes[0].required_columns = self.required_columns.clone(); - return Ok(self); + if projection_input + .as_any() + .downcast_ref::() + .is_some() + || projection_input + .as_any() + .downcast_ref::() + .is_some() + { + self = match self.try_swap_with_limit()? { + swapped if swapped.transformed => { + return Ok(swapped.data); + } + no_change => no_change.data, + }; } // Source providers: @@ -457,6 +450,32 @@ impl ProjectionOptimizer { } } + fn try_swap_with_limit(mut self) -> Result> { + let Some(projection) = self.plan.as_any().downcast_ref::() else { + return Ok(Transformed::no(self)); + }; + // 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() { + self.children_nodes[0].required_columns = self + .required_columns + .iter() + .flat_map(|e| collect_columns(&projection.expr()[e.index()].0)) + .collect::>(); + return Ok(Transformed::no(self.clone())); + } + + let new_p = self + .plan + .with_new_children(self.children_nodes[0].plan.children())?; + self.plan = self.children_nodes[0] + .plan + .clone() + .with_new_children(vec![new_p.clone()])?; + self.children_nodes[0].plan = new_p; + self.children_nodes[0].required_columns = self.required_columns.clone(); + return Ok(Transformed::yes(self)); + } + /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. fn try_projected_csv(self) -> Result { // These plans are known. From 4f33cf0b5ab37ddf9988ba76eb500f178067c35f Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 19 Apr 2024 10:29:50 +0300 Subject: [PATCH 61/85] Fixing all tests --- .../optimize_projections.rs | 568 +++++++++--------- datafusion/core/tests/sql/explain_analyze.rs | 2 +- .../sqllogictest/test_files/group_by.slt | 8 +- .../join_disable_repartition_joins.slt | 4 +- datafusion/sqllogictest/test_files/joins.slt | 12 +- .../sqllogictest/test_files/subquery.slt | 16 +- .../sqllogictest/test_files/tpch/q7.slt.part | 4 +- .../sqllogictest/test_files/tpch/q8.slt.part | 4 +- .../sqllogictest/test_files/tpch/q9.slt.part | 4 +- datafusion/sqllogictest/test_files/union.slt | 4 +- 10 files changed, 311 insertions(+), 315 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9a42f9ab7e0e..91e7c60a34e7 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -50,7 +50,6 @@ //! mapping, it can remove itself and assign new schema mapping to the new node, //! which was the projection's input formerly. -use std::collections::{HashMap, HashSet}; use std::mem; use std::sync::Arc; @@ -94,6 +93,7 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE 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 @@ -103,17 +103,18 @@ pub struct ProjectionOptimizer { /// The plan resides in the node pub plan: Arc, /// The node above expects it can reach these columns. - pub required_columns: HashSet, + 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: HashMap, + 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. -type ColumnRequirements = HashMap; +/// 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. @@ -121,8 +122,8 @@ impl ProjectionOptimizer { let children = plan.children(); Self { plan, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: children.into_iter().map(Self::new_default).collect(), } } @@ -209,6 +210,14 @@ impl ProjectionOptimizer { .as_any() .downcast_ref::() .is_some() + || projection_input + .as_any() + .downcast_ref::() + .is_some() + || projection_input + .as_any() + .downcast_ref::() + .is_some() { self = match self.try_swap_with_limit()? { swapped if swapped.transformed => { @@ -232,7 +241,7 @@ impl ProjectionOptimizer { self.required_columns .iter() .flat_map(|e| collect_columns(&projection_plan.expr()[e.index()].0)) - .collect::>() + .collect::>() } else { // If the method is used with a non-projection plan, we must sustain the execution safely. collect_columns_in_plan_schema(projection_input) @@ -353,7 +362,7 @@ impl ProjectionOptimizer { let new_projection_requires = self .required_columns .into_iter() - .map(|col| new_mapping.remove(&col).unwrap_or(col)) + .map(|col| new_mapping.swap_remove(&col).unwrap_or(col)) .collect(); Ok(Transformed::yes(ProjectionOptimizer { @@ -377,7 +386,9 @@ impl ProjectionOptimizer { }; // Collect all column indices from the given projection expressions. - let projection_index = collect_column_indices_in_proj_exprs(projection.expr()); + let mut projection_index = + collect_column_indices_in_proj_exprs(projection.expr()); + projection_index.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. @@ -389,7 +400,6 @@ impl ProjectionOptimizer { collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(Transformed::no(self)); } - let new_hash_join = Arc::new(hash_join.with_projection(Some(projection_index.clone()))?) as Arc; @@ -429,7 +439,7 @@ impl ProjectionOptimizer { Ok(Transformed::yes(Self { plan: new_hash_join, required_columns, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: self.children_nodes.swap_remove(0).children_nodes, })) } else { @@ -437,14 +447,14 @@ impl ProjectionOptimizer { let new_join_node = Self { plan: new_hash_join, required_columns, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: self.children_nodes.swap_remove(0).children_nodes, }; let plan = Arc::new(new_projection) as Arc; Ok(Transformed::no(Self { plan, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_join_node], })) } @@ -460,7 +470,7 @@ impl ProjectionOptimizer { .required_columns .iter() .flat_map(|e| collect_columns(&projection.expr()[e.index()].0)) - .collect::>(); + .collect::>(); return Ok(Transformed::no(self.clone())); } @@ -473,7 +483,7 @@ impl ProjectionOptimizer { .with_new_children(vec![new_p.clone()])?; self.children_nodes[0].plan = new_p; self.children_nodes[0].required_columns = self.required_columns.clone(); - return Ok(Transformed::yes(self)); + Ok(Transformed::yes(self)) } /// Tries to embed [`ProjectionExec`] into its input [`CsvExec`]. @@ -509,8 +519,8 @@ impl ProjectionOptimizer { csv.escape(), csv.file_compression_type, )) as _, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), // Sources cannot have a mapping. + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), // Sources cannot have a mapping. children_nodes: vec![], }) } else { @@ -533,8 +543,8 @@ impl ProjectionOptimizer { csv.escape(), csv.file_compression_type, )) as _, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), // Sources cannot have a mapping. + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), // Sources cannot have a mapping. children_nodes: vec![], }; @@ -665,7 +675,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -692,7 +702,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -730,7 +740,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -765,7 +775,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -805,7 +815,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -852,7 +862,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -894,7 +904,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -940,7 +950,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], } @@ -975,7 +985,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: new_children, } @@ -1008,7 +1018,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), // clear the requirements + required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: new_children, } @@ -1060,7 +1070,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping, children_nodes: vec![new_left_child, new_right_child], } @@ -1089,7 +1099,7 @@ impl ProjectionOptimizer { ); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -1114,14 +1124,14 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + 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 = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }) @@ -1231,8 +1241,8 @@ impl ProjectionOptimizer { )?; Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_left_node, new_right_node], }) } @@ -1275,8 +1285,8 @@ impl ProjectionOptimizer { Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_left_node, right_node], }) } @@ -1319,13 +1329,13 @@ impl ProjectionOptimizer { Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![left_node, new_right_node], }) } (true, true) => { - self.required_columns = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }); @@ -1409,8 +1419,8 @@ impl ProjectionOptimizer { )?; Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_left_node, new_right_node], }) } @@ -1453,8 +1463,8 @@ impl ProjectionOptimizer { Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_left_node, right_node], }) } @@ -1497,13 +1507,13 @@ impl ProjectionOptimizer { Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![left_node, new_right_node], }) } (true, true) => { - self.required_columns = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }); @@ -1578,8 +1588,8 @@ impl ProjectionOptimizer { )?; Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_left_node, new_right_node], }) } @@ -1618,8 +1628,8 @@ impl ProjectionOptimizer { Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![new_left_node, right_node], }) } @@ -1658,13 +1668,13 @@ impl ProjectionOptimizer { Ok(ProjectionOptimizer { plan: Arc::new(new_hash_join), - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + required_columns: IndexSet::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![left_node, new_right_node], }) } (true, true) => { - self.required_columns = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }); @@ -1731,7 +1741,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping, children_nodes: vec![new_left_child, new_right_child], } @@ -1742,7 +1752,7 @@ impl ProjectionOptimizer { let new_filter = update_non_equivalence_conditions( nlj.filter(), &analyzed_join_right, - &HashMap::new(), + &IndexMap::new(), ); let (new_left_child, mut left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; @@ -1766,7 +1776,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -1776,7 +1786,7 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let new_filter = update_non_equivalence_conditions( nlj.filter(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let (new_right_child, mut right_schema_mapping) = self @@ -1797,14 +1807,14 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + 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 = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }) @@ -1818,7 +1828,7 @@ impl ProjectionOptimizer { let new_filter = update_non_equivalence_conditions( nlj.filter(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let (new_left_child, left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; @@ -1838,7 +1848,7 @@ impl ProjectionOptimizer { .collect(); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -1861,7 +1871,7 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let new_filter = update_non_equivalence_conditions( nlj.filter(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self @@ -1882,7 +1892,7 @@ impl ProjectionOptimizer { .collect(); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: right_schema_mapping, children_nodes: vec![left_child, new_right_child], } @@ -1967,7 +1977,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping, children_nodes: vec![new_left_child, new_right_child], } @@ -1978,12 +1988,12 @@ impl ProjectionOptimizer { let new_on = update_equivalence_conditions( smj.on(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let new_filter = update_non_equivalence_conditions( smj.filter.as_ref(), &analyzed_join_right, - &HashMap::new(), + &IndexMap::new(), ); let (new_left_child, mut left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; @@ -2010,7 +2020,7 @@ impl ProjectionOptimizer { ); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -2020,12 +2030,12 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( smj.on(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let new_filter = update_non_equivalence_conditions( smj.filter.as_ref(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let (new_right_child, mut right_schema_mapping) = self @@ -2048,14 +2058,14 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + 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 = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }) @@ -2069,12 +2079,12 @@ impl ProjectionOptimizer { let new_on = update_equivalence_conditions( smj.on(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let new_filter = update_non_equivalence_conditions( smj.filter.as_ref(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let (new_left_child, left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; @@ -2097,7 +2107,7 @@ impl ProjectionOptimizer { .collect(); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -2120,12 +2130,12 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( smj.on(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let new_filter = update_non_equivalence_conditions( smj.filter.as_ref(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self @@ -2149,7 +2159,7 @@ impl ProjectionOptimizer { .collect(); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: right_schema_mapping, children_nodes: vec![left_child, new_right_child], } @@ -2245,7 +2255,7 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping, children_nodes: vec![new_left_child, new_right_child], } @@ -2256,12 +2266,12 @@ impl ProjectionOptimizer { let new_on = update_equivalence_conditions( shj.on(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let new_filter = update_non_equivalence_conditions( shj.filter(), &analyzed_join_right, - &HashMap::new(), + &IndexMap::new(), ); let (new_left_child, mut left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; @@ -2289,7 +2299,7 @@ impl ProjectionOptimizer { ); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -2299,12 +2309,12 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( shj.on(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let new_filter = update_non_equivalence_conditions( shj.filter(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let (new_right_child, mut right_schema_mapping) = self @@ -2329,14 +2339,14 @@ impl ProjectionOptimizer { self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + 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 = HashSet::new(); + self.required_columns = IndexSet::new(); self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan); }) @@ -2350,12 +2360,12 @@ impl ProjectionOptimizer { let new_on = update_equivalence_conditions( shj.on(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let new_filter = update_non_equivalence_conditions( shj.filter(), &analyzed_join_left, - &HashMap::new(), + &IndexMap::new(), ); let (new_left_child, left_schema_mapping) = self.insert_projection_below_left_child(analyzed_join_left)?; @@ -2380,7 +2390,7 @@ impl ProjectionOptimizer { .collect(); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: left_schema_mapping, children_nodes: vec![new_left_child, right_child], } @@ -2403,12 +2413,12 @@ impl ProjectionOptimizer { let mut left_child = self.children_nodes.swap_remove(0); let new_on = update_equivalence_conditions( shj.on(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let new_filter = update_non_equivalence_conditions( shj.filter(), - &HashMap::new(), + &IndexMap::new(), &analyzed_join_right, ); let (new_right_child, right_schema_mapping) = self @@ -2434,7 +2444,7 @@ impl ProjectionOptimizer { .collect(); self = ProjectionOptimizer { plan, - required_columns: HashSet::new(), + required_columns: IndexSet::new(), schema_mapping: right_schema_mapping, children_nodes: vec![left_child, new_right_child], } @@ -2466,14 +2476,14 @@ impl ProjectionOptimizer { .required_columns .iter() .map(|req_col| req_col.index()) - .collect::>(); + .collect::>(); let unused_aggr_exprs = agg .aggr_expr() .iter() .enumerate() .filter(|(idx, _expr)| !required_indices.contains(&(idx + group_columns_len))) .map(|(idx, _expr)| idx) - .collect::>(); + .collect::>(); if !unused_aggr_exprs.is_empty() { let new_plan = AggregateExec::try_new( @@ -2500,11 +2510,11 @@ impl ProjectionOptimizer { e.expressions() .iter() .flat_map(collect_columns) - .collect::>() + .collect::>() }), ); self.plan = Arc::new(new_plan); - self.required_columns = HashSet::new(); + self.required_columns = IndexSet::new(); } else { match agg.mode() { AggregateMode::Final | AggregateMode::FinalPartitioned => { @@ -2629,7 +2639,7 @@ impl ProjectionOptimizer { .map(|(window_expr, (_window_col, _usage))| { let new_exprs = update_expressions( &window_expr.expressions(), - &HashMap::new(), + &IndexMap::new(), ); window_expr.clone().with_new_expressions(new_exprs) }) @@ -2798,7 +2808,7 @@ impl ProjectionOptimizer { fn insert_projection( self, requirement_map: ColumnRequirements, - ) -> Result<(Self, HashMap)> { + ) -> Result<(Self, 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) = split_column_requirements(&requirement_map); @@ -2816,7 +2826,7 @@ impl ProjectionOptimizer { plan: inserted_projection, // Required columns must have been extended with self node requirements before this point. required_columns: new_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: self.children_nodes, }; Ok((inserted_projection, new_mapping)) @@ -2826,7 +2836,7 @@ impl ProjectionOptimizer { fn insert_multi_projection_below_union( self, requirement_map: ColumnRequirements, - ) -> Result<(Vec, HashMap)> { + ) -> 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) = split_column_requirements(&requirement_map); @@ -2860,7 +2870,7 @@ impl ProjectionOptimizer { .map(|(idx, (p, child))| ProjectionOptimizer { plan: p, required_columns: new_requirements[idx].clone(), - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![child], }) .collect(); @@ -2873,7 +2883,12 @@ impl ProjectionOptimizer { mut self, requirement_map_left: ColumnRequirements, requirement_map_right: ColumnRequirements, - ) -> Result<(Self, Self, HashMap, HashMap)> { + ) -> Result<( + Self, + Self, + IndexMap, + 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) = @@ -2888,7 +2903,7 @@ impl ProjectionOptimizer { let left_inserted_projection = ProjectionOptimizer { plan: inserted_projection, required_columns, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![self.children_nodes.swap_remove(0)], }; @@ -2904,7 +2919,7 @@ impl ProjectionOptimizer { let right_inserted_projection = ProjectionOptimizer { plan: inserted_projection, required_columns, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![self.children_nodes.swap_remove(0)], }; Ok(( @@ -2919,7 +2934,7 @@ impl ProjectionOptimizer { fn insert_projection_below_left_child( mut self, requirement_map_left: ColumnRequirements, - ) -> Result<(Self, HashMap)> { + ) -> Result<(Self, 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) = @@ -2934,7 +2949,7 @@ impl ProjectionOptimizer { let inserted_projection = ProjectionOptimizer { plan: inserted_projection, required_columns, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![self.children_nodes.swap_remove(0)], }; Ok((inserted_projection, new_mapping)) @@ -2944,7 +2959,7 @@ impl ProjectionOptimizer { fn insert_projection_below_right_child( mut self, requirement_map_right: ColumnRequirements, - ) -> Result<(Self, HashMap)> { + ) -> Result<(Self, 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) = @@ -2959,7 +2974,7 @@ impl ProjectionOptimizer { let inserted_projection = ProjectionOptimizer { plan: inserted_projection, required_columns, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![self.children_nodes.swap_remove(0)], }; Ok((inserted_projection, new_mapping)) @@ -2971,7 +2986,7 @@ impl ProjectionOptimizer { left_size: usize, requirement_map_left: ColumnRequirements, requirement_map_right: ColumnRequirements, - ) -> Result<(Self, Self, HashMap)> { + ) -> Result<(Self, Self, IndexMap)> { let original_right = self.children_nodes[1].plan.clone(); let ( new_left_child, @@ -3017,7 +3032,7 @@ impl ProjectionOptimizer { self, w_agg: &WindowAggExec, requirement_map: ColumnRequirements, - ) -> Result<(Self, HashMap, ColumnRequirements)> { + ) -> Result<(Self, IndexMap, ColumnRequirements)> { let original_schema_len = w_agg.schema().fields().len(); let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() @@ -3043,7 +3058,7 @@ impl ProjectionOptimizer { plan: inserted_projection, // Required columns must have been extended with self node requirements before this point. required_columns: new_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: self.children_nodes, }; Ok((inserted_projection, new_mapping, window)) @@ -3054,12 +3069,12 @@ impl ProjectionOptimizer { self, bw_agg: &BoundedWindowAggExec, requirement_map: ColumnRequirements, - ) -> Result<(Self, HashMap, ColumnRequirements)> { + ) -> Result<(Self, IndexMap, ColumnRequirements)> { let original_schema_len = bw_agg.schema().fields().len(); let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); - // let mut unused_columns = HashSet::new(); + // let mut unused_columns = IndexSet::new(); let (required_cols, mut unused_columns) = split_column_requirements(&base); let projected_exprs = convert_projection_exprs(required_cols); window.iter().for_each(|(col, used)| { @@ -3080,7 +3095,7 @@ impl ProjectionOptimizer { plan: inserted_projection, // Required columns must have been extended with self node requirements before this point. required_columns: new_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: self.children_nodes, }; Ok((inserted_projection, new_mapping, window)) @@ -3244,7 +3259,6 @@ impl ProjectionOptimizer { filter.schema().clone(), ) }); - match hj.join_type() { JoinType::Inner | JoinType::Left @@ -3256,7 +3270,11 @@ impl ProjectionOptimizer { .chain(right_mapping.iter().map(|(col1, col2)| { ( col1.index() - + hj.children()[0].schema().fields().len(), + + self.children_nodes[0] + .plan + .schema() + .fields() + .len(), col2.index() + self.children_nodes[0] .plan @@ -3265,7 +3283,7 @@ impl ProjectionOptimizer { .len(), ) })) - .collect::>(); + .collect::>(); let new_projection = projection .into_iter() .map(|idx| *index_mapping.get(&idx).unwrap_or(&idx)) @@ -3275,7 +3293,6 @@ impl ProjectionOptimizer { .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(), @@ -3291,33 +3308,12 @@ impl ProjectionOptimizer { hj.null_equals_null(), ) .map(|plan| Arc::new(plan) as _)?; - self.schema_mapping = left_mapping - .into_iter() - .chain(right_mapping.iter().map(|(col1, col2)| { - ( - Column::new( - col1.name(), - col1.index() - + hj.children()[0].schema().fields().len(), - ), - Column::new( - col2.name(), - col2.index() - + self.children_nodes[0] - .plan - .schema() - .fields() - .len(), - ), - ) - })) - .collect::>(); } JoinType::LeftSemi | JoinType::LeftAnti => { let index_mapping = left_mapping .iter() .map(|(col1, col2)| (col1.index(), col2.index())) - .collect::>(); + .collect::>(); let new_projection = projection .into_iter() .map(|idx| *index_mapping.get(&idx).unwrap_or(&idx)) @@ -3326,8 +3322,9 @@ impl ProjectionOptimizer { .first() .map(|first| *first != 0) .unwrap_or(true) - || !new_projection.windows(2).all(|w| w[0] + 1 == w[1]); - + || !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(), @@ -3349,7 +3346,7 @@ impl ProjectionOptimizer { let index_mapping = right_mapping .iter() .map(|(col1, col2)| (col1.index(), col2.index())) - .collect::>(); + .collect::>(); let mut new_projection = projection .into_iter() @@ -3360,7 +3357,9 @@ impl ProjectionOptimizer { .first() .map(|first| *first != 0) .unwrap_or(true) - || !new_projection.windows(2).all(|w| w[0] + 1 == w[1]); + || !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(), @@ -3393,7 +3392,7 @@ impl ProjectionOptimizer { Column::new(new.name(), new.index() + left_size), ) })) - .collect::>(); + .collect::>(); self.plan = rewrite_nested_loop_join( nlj, self.children_nodes[0].plan.clone(), @@ -3435,7 +3434,7 @@ impl ProjectionOptimizer { Column::new(new.name(), new.index() + left_size), ) })) - .collect::>(); + .collect::>(); self.plan = rewrite_sort_merge_join( smj, self.children_nodes[0].plan.clone(), @@ -3477,7 +3476,7 @@ impl ProjectionOptimizer { Column::new(new.name(), new.index() + left_size), ) })) - .collect::>(); + .collect::>(); self.plan = rewrite_symmetric_hash_join( shj, self.children_nodes[0].plan.clone(), @@ -3579,8 +3578,8 @@ impl ProjectionOptimizer { )?); let new_child = ProjectionOptimizer { plan: projection, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), + 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()])?; @@ -3628,7 +3627,7 @@ impl ProjectionOptimizer { let child_col_names = child_columns .iter() .map(|col| col.name().to_string()) - .collect::>(); + .collect::>(); if child_columns .iter() .all(|child_col| projection_columns.contains(child_col)) @@ -3694,7 +3693,6 @@ impl ConcreteTreeNode for ProjectionOptimizer { // 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) @@ -3727,13 +3725,11 @@ impl PhysicalOptimizerRule for OptimizeProjections { let mut optimized = optimizer.transform_down(&|o| { 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))?; - Ok(optimized.data.plan) } @@ -3754,7 +3750,7 @@ fn caching_projections( projection: &ProjectionExec, child_projection: &ProjectionExec, ) -> Result { - let mut column_ref_map: HashMap = HashMap::new(); + 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(&mut |expr| { @@ -3777,7 +3773,7 @@ fn caching_projections( /// a new projection is added to restore the initial column order and indices. fn satisfy_initial_schema( mut po: ProjectionOptimizer, - initial_requirements: HashSet, + initial_requirements: IndexSet, ) -> Result { if po.schema_mapping.is_empty() { // The initial schema is already satisfied, no further action required. @@ -3790,7 +3786,7 @@ fn satisfy_initial_schema( .into_iter() .map(|col| { // If there is a change, get the new index. - let final_column = po.schema_mapping.remove(&col).unwrap_or(col); + 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) @@ -3804,9 +3800,9 @@ fn satisfy_initial_schema( // Return a new ProjectionOptimizer with the final projection, resetting the schema mapping. Ok(ProjectionOptimizer { plan: final_projection, - required_columns: HashSet::new(), - schema_mapping: HashMap::new(), // Reset schema mapping as we've now satisfied the initial schema - children_nodes: vec![po], // Keep the original node as the child + 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 }) } } @@ -3814,7 +3810,7 @@ fn satisfy_initial_schema( /// 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 = HashMap::new(); + let mut requirement_map = IndexMap::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); @@ -3846,7 +3842,7 @@ fn analyze_requirements(node: &ProjectionOptimizer) -> ColumnRequirements { fn analyze_requirements_of_joins( left_child: &Arc, right_child: &Arc, - required_columns: &HashSet, + required_columns: &IndexSet, left_size: usize, ) -> (ColumnRequirements, ColumnRequirements) { let columns_in_schema = collect_columns_in_plan_schema(left_child) @@ -3865,15 +3861,15 @@ fn analyze_requirements_of_joins( (col, false) } }) - .collect::>(); + .collect::>(); let (requirement_map_left, mut requirement_map_right) = requirement_map .into_iter() - .partition::, _>(|(col, _)| col.index() < left_size); + .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::>(); + .collect::>(); (requirement_map_left, requirement_map_right) } @@ -3918,8 +3914,8 @@ fn preserve_requirements(po: ProjectionOptimizer) -> Result Ok(ProjectionOptimizer { plan: final_projection, required_columns: po.required_columns.clone(), - schema_mapping: HashMap::new(), // Reset schema mapping as we've now satisfied the initial schema - children_nodes: vec![po], // Keep the original node as the child + 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 }) } } @@ -3932,15 +3928,15 @@ fn all_columns_required(requirement_map: &ColumnRequirements) -> bool { /// Checks if all columns in the input schema are required by the projection. /// /// # Arguments -/// * `input_columns`: Reference to a `HashSet` representing the input columns. -/// * `projection_requires`: Reference to a `HashSet` representing the projection requirements. +/// * `input_columns`: Reference to a `IndexSet` representing the input columns. +/// * `projection_requires`: Reference to a `IndexSet` representing the projection requirements. /// /// # Returns /// `true` if all input columns are required, otherwise `false`. /// fn all_input_columns_required( - input_columns: &HashSet, - projection_requires: &HashSet, + input_columns: &IndexSet, + projection_requires: &IndexSet, ) -> bool { input_columns .iter() @@ -4001,10 +3997,10 @@ fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec, - unused_columns: &HashSet, -) -> HashMap { - let mut new_mapping = HashMap::new(); + 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() { @@ -4025,9 +4021,9 @@ fn calculate_column_mapping( /// Given a `ColumnRequirements`, it separates the required and redundant columns. fn split_column_requirements( requirements: &ColumnRequirements, -) -> (HashSet, HashSet) { - let mut required = HashSet::new(); - let mut unused = HashSet::new(); +) -> (IndexSet, IndexSet) { + let mut required = IndexSet::new(); + let mut unused = IndexSet::new(); for (col, is_req) in requirements { if *is_req { required.insert(col.clone()); @@ -4041,7 +4037,7 @@ fn split_column_requirements( /// 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: HashSet, + columns: IndexSet, ) -> Vec<(Arc, String)> { let mut new_expr = columns.into_iter().collect::>(); new_expr.sort_by_key(|column| column.index()); @@ -4055,11 +4051,11 @@ fn convert_projection_exprs( } fn extend_left_mapping_with_right( - mut left_schema_mapping: HashMap, + mut left_schema_mapping: IndexMap, right_child_plan: &Arc, left_size: usize, new_left_size: usize, -) -> HashMap { +) -> IndexMap { left_schema_mapping.extend( right_child_plan .schema() @@ -4072,7 +4068,7 @@ fn extend_left_mapping_with_right( Column::new(field.name(), new_left_size + idx), ) }) - .collect::>(), + .collect::>(), ); left_schema_mapping } @@ -4118,38 +4114,38 @@ fn removed_column_count( /// Then, projection inserts requirements into its child with these updated indices: "@0:b - @1:c - @2:a" /// /// # Arguments -/// * `requirements`: Reference to a `HashSet` representing the parent's column requirements. +/// * `requirements`: Reference to a `IndexSet` representing the parent's column requirements. /// * `projection_columns`: Slice of `Column` representing the column expressions in the projection. /// /// # Returns -/// A `HashSet` with updated column indices reflecting the child's perspective. +/// A `IndexSet` with updated column indices reflecting the child's perspective. /// fn map_parent_reqs_to_input_reqs( - requirements: &HashSet, + requirements: &IndexSet, projection_columns: &[Column], -) -> HashSet { +) -> IndexSet { requirements .iter() .map(|column| projection_columns[column.index()].clone()) - .collect::>() + .collect::>() } /// Calculates the index changes of columns after the removal of a projection. /// /// This function iterates through the columns and records the changes in their indices /// 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 and new index values in a hashmap. +/// projection and, if a change is observed, maps the old and new index values in a IndexMap. /// /// # Arguments /// * `columns` - A set of columns before the projection is removed. /// * `projection_columns` - A slice of columns as they appear in the projection. /// /// # Returns -/// A `HashMap` where the key is the original column and the value is the column with updated index. +/// A `IndexMap` where the key is the original column and the value is the column with updated index. fn index_changes_after_projection_removal( - columns: HashSet, + columns: IndexSet, projection_columns: &[Column], -) -> HashMap { +) -> IndexMap { columns .into_iter() .filter_map(|column| { @@ -4166,7 +4162,7 @@ fn index_changes_after_projection_removal( /// Filters the expressions of a [`ProjectionExec`] according to the given used column indices. fn collect_used_columns( projection_exprs: &[(Arc, String)], - used_columns: &HashSet, + used_columns: &IndexSet, ) -> Vec<(Arc, String)> { projection_exprs .iter() @@ -4187,9 +4183,9 @@ fn collect_used_columns( } fn collect_left_used_columns( - required_columns: HashSet, + required_columns: IndexSet, left_size: usize, -) -> HashSet { +) -> IndexSet { required_columns .into_iter() .filter(|col| col.index() < left_size) @@ -4213,14 +4209,14 @@ fn try_collect_alias_free_columns( Some(columns) } -/// Collects all fields of a schema from a given execution plan and converts them into a [`HashSet`] of [`Column`]. +/// Collects all fields of a schema from a given execution plan and converts them into a [`IndexSet`] of [`Column`]. /// /// # Arguments /// * `plan`: Reference to an Arc of an ExecutionPlan trait object. /// /// # Returns -/// A `HashSet` containing all columns from the plan's schema. -fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet { +/// A `IndexSet` containing all columns from the plan's schema. +fn collect_columns_in_plan_schema(plan: &Arc) -> IndexSet { plan.schema() .fields() .iter() @@ -4240,14 +4236,14 @@ fn collect_columns_in_plan_schema(plan: &Arc) -> HashSet` containing all columns from the join conditions. +/// A `IndexSet` containing all columns from the join conditions. fn collect_columns_in_join_conditions( on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, left_size: usize, join_left_schema: SchemaRef, join_right_schema: SchemaRef, -) -> HashSet { +) -> IndexSet { let equivalence_columns = on .iter() .flat_map(|(col_left, col_right)| { @@ -4259,7 +4255,7 @@ fn collect_columns_in_join_conditions( .collect_vec(); left_columns.into_iter().chain(right_columns).collect_vec() }) - .collect::>(); + .collect::>(); let non_equivalence_columns = filter .map(|filter| { filter @@ -4275,13 +4271,13 @@ fn collect_columns_in_join_conditions( col_idx.index + left_size, ), }) - .collect::>() + .collect::>() }) .unwrap_or_default(); equivalence_columns .into_iter() .chain(non_equivalence_columns) - .collect::>() + .collect::>() } /// Given expressions of a projection, the function collects all mentioned columns into a vector. @@ -4325,13 +4321,13 @@ fn split_column_indices( } fn collect_hj_left_requirements( - all_requirements: &HashSet, + all_requirements: &IndexSet, join_projection: &[usize], join_left_input_size: usize, join_left_schema: SchemaRef, on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, -) -> HashSet { +) -> IndexSet { let mut hj_left_requirements = all_requirements .iter() .filter_map(|req| { @@ -4341,11 +4337,11 @@ fn collect_hj_left_requirements( None } }) - .collect::>(); + .collect::>(); hj_left_requirements.extend( on.iter() .flat_map(|(left_on, _)| collect_columns(left_on)) - .collect::>(), + .collect::>(), ); hj_left_requirements.extend( filter @@ -4363,7 +4359,7 @@ fn collect_hj_left_requirements( None } }) - .collect::>() + .collect::>() }) .unwrap_or_default(), ); @@ -4374,12 +4370,12 @@ fn collect_right_hj_left_requirements( join_left_schema: SchemaRef, on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, -) -> HashSet { - let mut hj_left_requirements = HashSet::new(); +) -> IndexSet { + let mut hj_left_requirements = IndexSet::new(); hj_left_requirements.extend( on.iter() .flat_map(|(left_on, _)| collect_columns(left_on)) - .collect::>(), + .collect::>(), ); hj_left_requirements.extend( filter @@ -4397,7 +4393,7 @@ fn collect_right_hj_left_requirements( None } }) - .collect::>() + .collect::>() }) .unwrap_or_default(), ); @@ -4405,13 +4401,13 @@ fn collect_right_hj_left_requirements( } fn collect_hj_right_requirements( - all_requirements: &HashSet, + all_requirements: &IndexSet, join_projection: &[usize], join_left_input_size: usize, join_right_schema: SchemaRef, on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, -) -> HashSet { +) -> IndexSet { let mut hj_right_requirements = all_requirements .iter() .filter_map(|req| { @@ -4424,11 +4420,11 @@ fn collect_hj_right_requirements( None } }) - .collect::>(); + .collect::>(); hj_right_requirements.extend( on.iter() .flat_map(|(_, right_on)| collect_columns(right_on)) - .collect::>(), + .collect::>(), ); hj_right_requirements.extend( @@ -4447,7 +4443,7 @@ fn collect_hj_right_requirements( None } }) - .collect::>() + .collect::>() }) .unwrap_or_default(), ); @@ -4455,20 +4451,20 @@ fn collect_hj_right_requirements( } fn collect_right_hj_right_requirements( - all_requirements: &HashSet, + all_requirements: &IndexSet, join_projection: &[usize], join_right_schema: SchemaRef, on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, -) -> HashSet { +) -> IndexSet { let mut hj_right_requirements = all_requirements .iter() .map(|req| Column::new(req.name(), join_projection[req.index()])) - .collect::>(); + .collect::>(); hj_right_requirements.extend( on.iter() .flat_map(|(_, right_on)| collect_columns(right_on)) - .collect::>(), + .collect::>(), ); hj_right_requirements.extend( @@ -4487,7 +4483,7 @@ fn collect_right_hj_right_requirements( None } }) - .collect::>() + .collect::>() }) .unwrap_or_default(), ); @@ -4570,7 +4566,7 @@ fn update_expr_with_projection( /// Rewrites the expressions with new index values. fn update_expressions( exprs: &[Arc], - mapping: &HashMap, + mapping: &IndexMap, ) -> Vec> { exprs .iter() @@ -4581,7 +4577,7 @@ fn update_expressions( /// Rewrites the sort expressions with new index values. fn update_sort_expressions( sort_exprs: &[PhysicalSortExpr], - mapping: &HashMap, + mapping: &IndexMap, ) -> LexOrdering { sort_exprs .iter() @@ -4596,7 +4592,7 @@ fn update_sort_expressions( /// the mapping. If it is not a hash partitioning, they remains as they are. fn update_partitioning_expressions( partitioning: &Partitioning, - mapping: &HashMap, + mapping: &IndexMap, ) -> Partitioning { if let Partitioning::Hash(exprs, size) = partitioning { let updated_exprs = update_expressions(exprs, mapping); @@ -4609,7 +4605,7 @@ fn update_partitioning_expressions( /// Rewrites the window expressions with new index values. fn update_window_exprs( window_exprs: &[Arc], - mapping: &HashMap, + mapping: &IndexMap, ) -> Option>> { window_exprs .iter() @@ -4623,7 +4619,7 @@ fn update_window_exprs( /// Rewrites the aggregate expressions with new index values. fn update_aggregate_exprs( aggregate_exprs: &[Arc], - mapping: &HashMap, + mapping: &IndexMap, ) -> Option>> { aggregate_exprs .iter() @@ -4642,8 +4638,8 @@ fn update_aggregate_exprs( /// Rewrites the expressions in equivalence condition of a join with new index values. fn update_join_on( join_on: JoinOnRef, - left_mapping: &HashMap, - right_mapping: &HashMap, + left_mapping: &IndexMap, + right_mapping: &IndexMap, ) -> JoinOn { join_on .iter() @@ -4660,7 +4656,7 @@ fn update_join_on( /// it updates the indices of columns in the [`PhysicalExpr`]. fn update_column_index( expr: &Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Arc { let mut state = RewriteState::Unchanged; let new_expr = expr @@ -4774,9 +4770,9 @@ fn update_non_equivalence_conditions( } fn update_right_child_requirements( - required_columns: &HashSet, + required_columns: &IndexSet, left_size: usize, -) -> HashSet { +) -> IndexSet { required_columns .iter() .filter(|col| col.index() >= left_size) @@ -4786,7 +4782,7 @@ fn update_right_child_requirements( fn update_mapping( node: &mut ProjectionOptimizer, - mut child_mappings: Vec>, + mut child_mappings: Vec>, ) { if node.schema_mapping.is_empty() { node.schema_mapping = child_mappings.swap_remove(0); @@ -4807,7 +4803,7 @@ fn update_mapping( fn update_mapping_cross( node: &mut ProjectionOptimizer, - mut child_mappings: Vec>, + mut child_mappings: Vec>, ) { if node.schema_mapping.is_empty() { node.schema_mapping = child_mappings.swap_remove(0); @@ -4851,7 +4847,7 @@ fn update_mapping_cross( if let Some((match_i, _match_f)) = self_node_map.clone().iter().find(|(_i, f)| initial == **f) { - self_node_map.remove(match_i); + self_node_map.swap_remove(match_i); (match_i.clone(), new) } else { (initial, new) @@ -4863,9 +4859,9 @@ fn update_mapping_cross( } fn update_right_mapping( - right_schema_mapping: HashMap, + right_schema_mapping: IndexMap, left_size: usize, -) -> HashMap { +) -> IndexMap { right_schema_mapping .into_iter() .map(|(old, new)| { @@ -4878,9 +4874,9 @@ fn update_right_mapping( } fn update_right_requirements( - required_columns: HashSet, + required_columns: IndexSet, left_size: usize, -) -> HashSet { +) -> IndexSet { required_columns .into_iter() .map(|col| Column::new(col.name(), col.index() + left_size)) @@ -4918,8 +4914,8 @@ fn update_proj_exprs( } fn update_hj_children( - hj_left_requirements: &HashSet, - hj_right_requirements: &HashSet, + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, mut children: Vec, hj: &HashJoinExec, ) -> Result<(ProjectionOptimizer, ProjectionOptimizer)> { @@ -4940,7 +4936,7 @@ fn update_hj_children( let new_left_node = ProjectionOptimizer { plan: new_left_projection_arc, required_columns: new_left_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![children.swap_remove(0)], }; @@ -4964,7 +4960,7 @@ fn update_hj_children( let new_right_node = ProjectionOptimizer { plan: new_right_projection_arc, required_columns: new_right_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![children.swap_remove(0)], }; @@ -4972,8 +4968,8 @@ fn update_hj_children( } fn update_hj_left_child( - hj_left_requirements: &HashSet, - hj_right_requirements: &HashSet, + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, mut children: Vec, hj: &HashJoinExec, ) -> Result<(ProjectionOptimizer, ProjectionOptimizer)> { @@ -4994,7 +4990,7 @@ fn update_hj_left_child( let new_left_node = ProjectionOptimizer { plan: new_left_projection_arc, required_columns: new_left_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![children.swap_remove(0)], }; @@ -5005,8 +5001,8 @@ fn update_hj_left_child( } fn update_hj_right_child( - hj_left_requirements: &HashSet, - hj_right_requirements: &HashSet, + hj_left_requirements: &IndexSet, + hj_right_requirements: &IndexSet, mut children: Vec, hj: &HashJoinExec, ) -> Result<(ProjectionOptimizer, ProjectionOptimizer)> { @@ -5029,7 +5025,7 @@ fn update_hj_right_child( let new_right_node = ProjectionOptimizer { plan: new_right_projection_arc, required_columns: new_right_requirements, - schema_mapping: HashMap::new(), + schema_mapping: IndexMap::new(), children_nodes: vec![children.swap_remove(1)], }; @@ -5040,32 +5036,32 @@ fn update_hj_right_child( } fn update_hj_children_mapping( - hj_left_requirements: &HashSet, - hj_right_requirements: &HashSet, -) -> (HashMap, HashMap) { + 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::>(); + .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::>(); + .collect::>(); (left_mapping, right_mapping) } fn update_hj_projection( projection: Option>, hj_left_schema: SchemaRef, - hj_left_requirements: HashSet, - left_mapping: HashMap, - right_mapping: HashMap, + hj_left_requirements: IndexSet, + left_mapping: IndexMap, + right_mapping: IndexMap, join_left_input_size: usize, ) -> Option> { projection.map(|projection| { @@ -5095,7 +5091,7 @@ fn update_hj_projection( fn update_hj_projection_right( projection: Option>, - right_mapping: HashMap, + right_mapping: IndexMap, ) -> Option> { projection.map(|projection| { projection @@ -5119,14 +5115,14 @@ fn update_hj_projection_right( /// # Arguments /// * `predicate` - The predicate expression of the filter. /// * `input_plan` - The input execution plan on which the filter is applied. -/// * `mapping` - A hashmap with old and new column index mappings. +/// * `mapping` - A IndexMap with old and new column index mappings. /// /// # Returns /// A `Result` containing the new `FilterExec` wrapped in an `Arc`. fn rewrite_filter( predicate: &Arc, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result> { FilterExec::try_new(update_column_index(predicate, mapping), input_plan) .map(|plan| Arc::new(plan) as _) @@ -5134,8 +5130,8 @@ fn rewrite_filter( fn rewrite_hj_filter( filter: Option<&JoinFilter>, - left_mapping: &HashMap, - right_mapping: &HashMap, + left_mapping: &IndexMap, + right_mapping: &IndexMap, ) -> Option { filter.map(|filter| { JoinFilter::new( @@ -5178,7 +5174,7 @@ fn rewrite_hj_filter( /// # Arguments /// * `projection` - The original projection execution plan. /// * `input_plan` - The input execution plan on which the projection is applied. -/// * `mapping` - A hashmap with old and new column index mappings. +/// * `mapping` - A IndexMap with old and new column index mappings. /// /// # Returns /// A `Result` containing the new `ProjectionExec` wrapped in an `Arc`. @@ -5186,7 +5182,7 @@ fn rewrite_hj_filter( fn rewrite_projection( projection: &ProjectionExec, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result> { ProjectionExec::try_new( projection @@ -5207,14 +5203,14 @@ fn rewrite_projection( /// # Arguments /// * `partitioning` - The original partitioning strategy. /// * `input_plan` - The input execution plan on which repartitioning is applied. -/// * `mapping` - A hashmap with old and new column index mappings. +/// * `mapping` - A IndexMap with old and new column index mappings. /// /// # Returns /// A `Result` containing the new `RepartitionExec` wrapped in an `Arc`. fn rewrite_repartition( partitioning: &Partitioning, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result> { let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { let new_exprs = update_expressions(exprs, mapping); @@ -5233,14 +5229,14 @@ fn rewrite_repartition( /// # Arguments /// * `sort` - The original sort execution plan. /// * `input_plan` - The input execution plan on which sorting is applied. -/// * `mapping` - A hashmap with old and new column index mappings. +/// * `mapping` - A IndexMap with old and new column index mappings. /// /// # Returns /// A `Result` containing the new `SortExec` wrapped in an `Arc`. fn rewrite_sort( sort: &SortExec, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result> { let new_sort_exprs = update_sort_expressions(sort.expr(), mapping); Ok(Arc::new( @@ -5257,14 +5253,14 @@ fn rewrite_sort( /// # Arguments /// * `sort` - The original `SortPreservingMergeExec` plan. /// * `input_plan` - The input execution plan to which the sort preserving merge is applied. -/// * `mapping` - A hashmap with old and new column index mappings. +/// * `mapping` - A IndexMap with old and new column index mappings. /// /// # Returns /// A `Result` containing the new `SortPreservingMergeExec` wrapped in an `Arc`. fn rewrite_sort_preserving_merge( sort: &SortPreservingMergeExec, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result> { let new_sort_exprs = update_sort_expressions(sort.expr(), mapping); Ok(Arc::new( @@ -5276,8 +5272,8 @@ fn rewrite_nested_loop_join( nlj: &NestedLoopJoinExec, left_input_plan: Arc, right_input_plan: Arc, - left_mapping: &HashMap, - right_mapping: &HashMap, + left_mapping: &IndexMap, + right_mapping: &IndexMap, left_size: usize, ) -> Result> { let new_filter = nlj.filter().map(|filter| { @@ -5339,8 +5335,8 @@ fn rewrite_sort_merge_join( smj: &SortMergeJoinExec, left_input_plan: Arc, right_input_plan: Arc, - left_mapping: &HashMap, - right_mapping: &HashMap, + left_mapping: &IndexMap, + right_mapping: &IndexMap, left_size: usize, ) -> Result> { let new_on = update_join_on(smj.on(), left_mapping, right_mapping); @@ -5406,8 +5402,8 @@ fn rewrite_symmetric_hash_join( shj: &SymmetricHashJoinExec, left_input_plan: Arc, right_input_plan: Arc, - left_mapping: &HashMap, - right_mapping: &HashMap, + left_mapping: &IndexMap, + right_mapping: &IndexMap, left_size: usize, ) -> Result> { let new_on = update_join_on(shj.on(), left_mapping, right_mapping); @@ -5490,7 +5486,7 @@ fn rewrite_symmetric_hash_join( fn rewrite_aggregate( agg: &AggregateExec, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result>> { let new_group_by = PhysicalGroupBy::new( agg.group_expr() @@ -5545,7 +5541,7 @@ fn rewrite_aggregate( fn rewrite_window_aggregate( w_agg: &WindowAggExec, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result>> { let new_window = if let Some(new_window) = update_window_exprs(w_agg.window_expr(), mapping) { @@ -5572,7 +5568,7 @@ fn rewrite_window_aggregate( fn rewrite_bounded_window_aggregate( bw_agg: &BoundedWindowAggExec, input_plan: Arc, - mapping: &HashMap, + mapping: &IndexMap, ) -> Result>> { let new_window = if let Some(new_window) = update_window_exprs(bw_agg.window_expr(), mapping) { @@ -6067,9 +6063,9 @@ mod tests { OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@2 as d]", - " CoalescePartitionsExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, d], has_header=false", + "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); @@ -6407,10 +6403,10 @@ mod tests { // 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@0 as c_from_left, b@1 as b_from_left, a@2 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=[c@2, b@1, a@0, 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"]; + "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( diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index f8f815a1a38d..4ae69cfc5d5d 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -658,7 +658,7 @@ 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", + "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", diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index b44afc0974e8..9c24039f4f41 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] 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 60a14f78bdf5..f33a04ce98c9 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 cf7dc5b45296..4c251653707a 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1451,9 +1451,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@2 as t1_name, t1_int@3 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1477,9 +1477,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@2 as t1_name, t1_int@3 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 9297fd7ef6d3..3f42c7b5dab8 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -187,8 +187,8 @@ 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 +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 @@ -220,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))] @@ -254,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 @@ -291,8 +291,8 @@ 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 +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 diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index f699c3ee6734..99d5c934c9ad 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -91,8 +91,8 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] -08)--------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] -09)----------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 1da304b3f946..506c68c83f5b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -97,8 +97,8 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] -08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] -09)----------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index e3e6a4d0e0d7..687926013ad4 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -84,8 +84,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -09)----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 550aebef0bcd..6b0d53fb124f 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -290,8 +290,8 @@ physical_plan 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] -16)----CoalesceBatchesExec: target_batch_size=2 +15)--CoalesceBatchesExec: target_batch_size=2 +16)----ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] 18)--------CoalesceBatchesExec: target_batch_size=2 19)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 From 679b7549a2154696303815021e06ab31b4c5a10a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 19 Apr 2024 13:32:24 +0300 Subject: [PATCH 62/85] Self review part 1 --- .../optimize_projections.rs | 3379 ++++++++--------- 1 file changed, 1686 insertions(+), 1693 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 91e7c60a34e7..bcd23878385b 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -154,7 +154,7 @@ impl ProjectionOptimizer { /// 1. If the input plan is also a projection, they can be merged into one projection. /// 2. The projection can be removed. /// 3. The projection can get narrower. - /// 4. The projection can be embedded into the plan below. + /// 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]; @@ -202,24 +202,9 @@ impl ProjectionOptimizer { }; } - if projection_input - .as_any() - .downcast_ref::() - .is_some() - || projection_input - .as_any() - .downcast_ref::() - .is_some() - || projection_input - .as_any() - .downcast_ref::() - .is_some() - || projection_input - .as_any() - .downcast_ref::() - .is_some() - { - self = match self.try_swap_with_limit()? { + // These kind of plans can swap the order with projections without any further modification. + if is_schema_agnostic(projection_input) { + self = match self.try_swap_trivial()? { swapped if swapped.transformed => { return Ok(swapped.data); } @@ -228,6 +213,8 @@ impl ProjectionOptimizer { } // 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(); } @@ -238,10 +225,10 @@ impl ProjectionOptimizer { self.children_nodes[0].required_columns = if let Some(projection_plan) = self.plan.as_any().downcast_ref::() { - self.required_columns - .iter() - .flat_map(|e| collect_columns(&projection_plan.expr()[e.index()].0)) - .collect::>() + 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) @@ -263,12 +250,13 @@ impl ProjectionOptimizer { 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) = @@ -314,9 +302,8 @@ impl ProjectionOptimizer { // 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); - - // If all fields of the input are necessary, we can remove the projection. let input_columns = collect_columns_in_plan_schema(projection_exec.input()); + // If 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, @@ -345,13 +332,16 @@ impl ProjectionOptimizer { }; let requirement_map = analyze_requirements(&self); - let (used_columns, unused_columns) = split_column_requirements(&requirement_map); + 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() { - // All projected items are used. return Ok(Transformed::no(self)); } + let projected_exprs = collect_used_columns(projection_exec.expr(), &used_columns); - let mut new_mapping = + let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); let new_projection_plan = Arc::new(ProjectionExec::try_new( @@ -359,10 +349,11 @@ impl ProjectionOptimizer { 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.swap_remove(&col).unwrap_or(col)) + .map(|col| new_mapping.get(&col).cloned().unwrap_or(col)) .collect(); Ok(Transformed::yes(ProjectionOptimizer { @@ -460,29 +451,32 @@ impl ProjectionOptimizer { } } - fn try_swap_with_limit(mut self) -> Result> { + /// 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, we should not try to push it down: if projection.expr().len() >= projection.input().schema().fields().len() { - self.children_nodes[0].required_columns = self - .required_columns - .iter() - .flat_map(|e| collect_columns(&projection.expr()[e.index()].0)) - .collect::>(); - return Ok(Transformed::no(self.clone())); + return Ok(Transformed::no(self)); } - let new_p = self + // New child of the projection is its input child. + let updated_projection = self .plan .with_new_children(self.children_nodes[0].plan.children())?; + // The child of the projection is now parent of the projection. self.plan = self.children_nodes[0] .plan .clone() - .with_new_children(vec![new_p.clone()])?; - self.children_nodes[0].plan = new_p; - self.children_nodes[0].required_columns = self.required_columns.clone(); + .with_new_children(vec![updated_projection.clone()])?; + + self.children_nodes[0].plan = updated_projection; + // Move the requirements without change. + self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); + Ok(Transformed::yes(self)) } @@ -2811,7 +2805,9 @@ impl ProjectionOptimizer { ) -> Result<(Self, 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) = split_column_requirements(&requirement_map); + let (used_columns, unused_columns) = + partition_column_requirements(requirement_map); + let projected_exprs = convert_projection_exprs(used_columns); let inserted_projection = Arc::new(ProjectionExec::try_new( projected_exprs, @@ -2839,7 +2835,8 @@ impl ProjectionOptimizer { ) -> 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) = split_column_requirements(&requirement_map); + let (used_columns, unused_columns) = + partition_column_requirements(requirement_map); let projected_exprs = convert_projection_exprs(used_columns); let inserted_projections = self @@ -2892,7 +2889,7 @@ impl ProjectionOptimizer { // 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) = - split_column_requirements(&requirement_map_left); + partition_column_requirements(requirement_map_left); let child_plan = self.plan.children().remove(0); let new_left_mapping = calculate_column_mapping(&used_columns, &unused_columns); let projected_exprs = convert_projection_exprs(used_columns); @@ -2908,7 +2905,7 @@ impl ProjectionOptimizer { }; let (used_columns, unused_columns) = - split_column_requirements(&requirement_map_right); + partition_column_requirements(requirement_map_right); let child_plan = self.plan.children().remove(1); let new_right_mapping = calculate_column_mapping(&used_columns, &unused_columns); let projected_exprs = convert_projection_exprs(used_columns); @@ -2938,7 +2935,7 @@ impl ProjectionOptimizer { // 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) = - split_column_requirements(&requirement_map_left); + partition_column_requirements(requirement_map_left); let child_plan = self.plan.children().remove(0); let new_mapping = calculate_column_mapping(&used_columns, &unused_columns); let projected_exprs = convert_projection_exprs(used_columns); @@ -2963,7 +2960,7 @@ impl ProjectionOptimizer { // 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) = - split_column_requirements(&requirement_map_right); + partition_column_requirements(requirement_map_right); let child_plan = self.plan.children().remove(1); let new_mapping = calculate_column_mapping(&used_columns, &unused_columns); let projected_exprs = convert_projection_exprs(used_columns); @@ -3037,7 +3034,7 @@ impl ProjectionOptimizer { let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); - let (used_columns, mut unused_columns) = split_column_requirements(&base); + let (used_columns, mut unused_columns) = partition_column_requirements(base); let projected_exprs = convert_projection_exprs(used_columns); window.iter().for_each(|(col, used)| { @@ -3075,7 +3072,7 @@ impl ProjectionOptimizer { .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); // let mut unused_columns = IndexSet::new(); - let (required_cols, mut unused_columns) = split_column_requirements(&base); + let (required_cols, mut unused_columns) = partition_column_requirements(base); let projected_exprs = convert_projection_exprs(required_cols); window.iter().for_each(|(col, used)| { if !used { @@ -3742,1008 +3739,960 @@ impl PhysicalOptimizerRule for OptimizeProjections { } } -// 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(&mut |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 +/// 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::>() } -/// 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 = IndexMap::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 +fn collect_left_used_columns( + required_columns: IndexSet, + left_size: usize, +) -> IndexSet { + required_columns + .into_iter() + .filter(|col| col.index() < left_size) + .collect() } -/// Analyzes the column requirements for join operations between left and right children plans. +/// Collects all fields of a schema from a given execution plan and converts them into a [`IndexSet`] of [`Column`]. /// -/// 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. +/// # Arguments +/// * `plan`: Reference to an Arc of an ExecutionPlan trait object. /// -/// The caller must ensure that the join node extends its requirements if the node's plan can introduce new columns. -/// Each column in the requirement maps corresponds to its own table schema index, not to the join output schema. +/// # Returns +/// A `IndexSet` containing all columns from the plan's schema. +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. /// /// # 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. +/// * `on`: Slice of tuples representing the equivalence conditions between columns. +/// * `filter`: Optional reference to a JoinFilter for non-equivalence conditions. +/// * `left_size`: The number of columns in the left table. +/// * `join_left_schema`: Schema reference of the left input to the join. +/// * `join_right_schema`: Schema reference of the right input to the join. /// /// # 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, +/// A `IndexSet` containing all columns from the join conditions. +fn collect_columns_in_join_conditions( + on: &[(PhysicalExprRef, PhysicalExprRef)], + filter: Option<&JoinFilter>, 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) + 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)), - ); - let requirement_map = columns_in_schema - .into_iter() - .map(|col| { - if required_columns.contains(&col) { - (col, true) - } else { - (col, false) - } + .map(|col| Column::new(col.name(), col.index() + left_size)) + .collect_vec(); + left_columns.into_iter().chain(right_columns).collect_vec() }) - .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 + .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() - .map(|(col, used)| (Column::new(col.name(), col.index() - left_size), used)) - .collect::>(); - (requirement_map_left, requirement_map_right) + .chain(non_equivalence_columns) + .collect::>() } -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 - }) - } -} - -/// 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) +/// 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::>() } -/// Checks if all columns in the input schema are required by the projection. -/// -/// # Arguments -/// * `input_columns`: Reference to a `IndexSet` representing the input columns. -/// * `projection_requires`: Reference to a `IndexSet` representing the projection requirements. -/// -/// # Returns -/// `true` if all input columns are required, otherwise `false`. -/// -fn all_input_columns_required( - input_columns: &IndexSet, - projection_requires: &IndexSet, -) -> bool { - input_columns +/// 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 .iter() - .all(|input_column| projection_requires.contains(input_column)) + .flat_map(|e| collect_columns(&projection_exprs[e.index()].0)) + .collect::>() } -fn window_agg_required( - original_schema_len: usize, - requirements: &ColumnRequirements, -) -> bool { - requirements +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(|(column, _used)| column.index() >= original_schema_len) - .any(|(_column, used)| *used) + .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 } -/// 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() +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 } -/// 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() -} +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::>(), + ); -/// 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()) + 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 } -/// Tries to rewrite the [`WindowExpr`] with the existing expressions to keep on optimization. -fn is_window_expr_rewritable(window_expr: &[Arc<(dyn WindowExpr)>]) -> bool { - window_expr.iter().all(|expr| expr.expressions().is_empty()) +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 } -/// Updates a source provider's projected columns according to the given -/// projection operator's expressions. To use this function safely, one must -/// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec { - projection.iter().map(|col| source[col.index()]).collect() +/// 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) } -/// 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 `ColumnRequirements`, it separates the required and redundant columns. -fn split_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.clone()); - } else { - unused.insert(col.clone()); - } - } - (required, unused) -} - -/// 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. -/// -/// # Arguments -/// * `requirement_map`: Reference to a ColumnRequirements map. -/// * `column_index`: The index of the column in question. -/// -/// # Returns -/// The number of removed columns before the 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 +#[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, } -/// 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" -/// -/// # Arguments -/// * `requirements`: Reference to a `IndexSet` representing the parent's column requirements. -/// * `projection_columns`: Slice of `Column` representing the column expressions in the projection. -/// -/// # Returns -/// A `IndexSet` with updated column indices reflecting the child's perspective. +/// The function operates in two modes: /// -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. +/// 1. When `sync_with_child` is `true`: /// -/// This function iterates through the columns and records the changes in their indices -/// 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 and new index values in a IndexMap. +/// 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`. /// -/// # Arguments -/// * `columns` - A set of columns before the projection is removed. -/// * `projection_columns` - A slice of columns as they appear in the projection. +/// 2. When `sync_with_child` is `false`: /// -/// # Returns -/// A `IndexMap` where the key is the original column and the value is the column with updated index. -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())) +/// 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_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::no(expr)); + } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + if sync_with_child { + state = RewriteState::RewrittenValid; + // Update the index of `column`: + Ok(Transformed::yes(projected_exprs[column.index()].0.clone())) } else { - None + // 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)), + ) } - }) - .collect() + }); + new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e.data)) } -/// 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)> { - projection_exprs +/// Rewrites the expressions with new index values. +fn update_expressions( + exprs: &[Arc], + mapping: &IndexMap, +) -> Vec> { + exprs .iter() - .enumerate() - .filter_map(|(idx, (expr, alias))| { - if used_columns - .iter() - .map(|column| column.index()) - .collect::>() - .contains(&idx) - { - Some((expr.clone(), alias.clone())) - } else { - None - } - }) + .map(|expr| update_column_index(expr, mapping)) .collect::>() } -fn collect_left_used_columns( - required_columns: IndexSet, - left_size: usize, -) -> IndexSet { - required_columns - .into_iter() - .filter(|col| col.index() < left_size) - .collect() +/// Rewrites the sort expressions with new index values. +fn update_sort_expressions( + sort_exprs: &[PhysicalSortExpr], + mapping: &IndexMap, +) -> LexOrdering { + sort_exprs + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, mapping), + options: sort_expr.options, + }) + .collect::>() } -/// 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, - } +/// Updates the expressions subject to hashing of the `Partitioning` according to +/// the mapping. If it is not a hash partitioning, they remains as they are. +fn update_partitioning_expressions( + partitioning: &Partitioning, + mapping: &IndexMap, +) -> Partitioning { + if let Partitioning::Hash(exprs, size) = partitioning { + let updated_exprs = update_expressions(exprs, mapping); + Partitioning::Hash(updated_exprs, *size) + } else { + partitioning.clone() } - Some(columns) } -/// Collects all fields of a schema from a given execution plan and converts them into a [`IndexSet`] of [`Column`]. -/// -/// # Arguments -/// * `plan`: Reference to an Arc of an ExecutionPlan trait object. -/// -/// # Returns -/// A `IndexSet` containing all columns from the plan's schema. -fn collect_columns_in_plan_schema(plan: &Arc) -> IndexSet { - plan.schema() - .fields() +/// Rewrites the window expressions with new index values. +fn update_window_exprs( + window_exprs: &[Arc], + mapping: &IndexMap, +) -> Option>> { + window_exprs .iter() - .enumerate() - .map(|(i, f)| Column::new(f.name(), i)) - .collect() + .map(|window_expr| { + let new_exprs = update_expressions(&window_expr.expressions(), mapping); + window_expr.clone().with_new_expressions(new_exprs) + }) + .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. -/// -/// # Arguments -/// * `on`: Slice of tuples representing the equivalence conditions between columns. -/// * `filter`: Optional reference to a JoinFilter for non-equivalence conditions. -/// * `left_size`: The number of columns in the left table. -/// * `join_left_schema`: Schema reference of the left input to the join. -/// * `join_right_schema`: Schema reference of the right input to the join. -/// -/// # Returns -/// A `IndexSet` containing all columns from the join conditions. -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 +/// Rewrites the aggregate expressions with new index values. +fn update_aggregate_exprs( + aggregate_exprs: &[Arc], + mapping: &IndexMap, +) -> Option>> { + aggregate_exprs .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::>() + .map(|aggr_expr| { + aggr_expr.clone().with_new_expressions( + aggr_expr + .expressions() + .iter() + .map(|expr| update_column_index(expr, mapping)) + .collect(), + ) }) - .unwrap_or_default(); - equivalence_columns - .into_iter() - .chain(non_equivalence_columns) - .collect::>() + .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 +/// 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 { + join_on .iter() - .flat_map(|(expr, _)| collect_columns(expr)) - .map(|col| col.index()) - .collect::>() + .map(|(left, right)| { + ( + update_column_index(left, left_mapping), + update_column_index(right, right_mapping), + ) + }) + .collect() } -fn split_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 +/// Given mapping representing the initial and new index values, +/// it updates the indices of columns in the [`PhysicalExpr`]. +fn update_column_index( + expr: &Arc, + mapping: &IndexMap, +) -> Arc { + let mut state = RewriteState::Unchanged; + let new_expr = expr + .clone() + .transform_up_mut(&mut |expr: Arc| { + if state == RewriteState::RewrittenInvalid { + return Ok(Transformed::no(expr)); } - }) - .collect::>(); - let unused_indices = file_scan_projection - .into_iter() - .enumerate() - .filter_map(|(idx, csv_idx)| { - if used_indices.contains(&csv_idx) { - None + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + state = RewriteState::RewrittenValid; + // Update the index of `column`: + if let Some(updated) = mapping.get(column) { + Ok(Transformed::yes(Arc::new(updated.clone()) as _)) } else { - Some(idx) + Ok(Transformed::no(expr.clone())) } }) - .collect::>(); - (used_indices, unused_indices) + .unwrap() + .data; + new_expr } -fn collect_hj_left_requirements( - all_requirements: &IndexSet, - join_projection: &[usize], - join_left_input_size: usize, - join_left_schema: SchemaRef, +/// Updates the equivalence conditions of the joins according to the new indices of columns. +fn update_equivalence_conditions( 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 + requirement_map_left: &ColumnRequirements, + requirement_map_right: &ColumnRequirements, +) -> JoinOn { + on.iter() + .map(|(left_col, right_col)| { + let mut left_state = RewriteState::Unchanged; + let mut right_state = RewriteState::Unchanged; + ( + left_col + .clone() + .transform_up_mut(&mut |expr: Arc| { + if left_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::no(expr)); } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + left_state = RewriteState::RewrittenValid; + Ok(Transformed::yes(Arc::new(Column::new( + column.name(), + column.index() + - removed_column_count( + requirement_map_left, + column.index(), + ), + )))) }) - .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 + .unwrap() + .data, + right_col + .clone() + .transform_up_mut(&mut |expr: Arc| { + if right_state == RewriteState::RewrittenInvalid { + return Ok(Transformed::no(expr)); } + let Some(column) = expr.as_any().downcast_ref::() else { + return Ok(Transformed::no(expr)); + }; + right_state = RewriteState::RewrittenValid; + Ok(Transformed::yes(Arc::new(Column::new( + column.name(), + column.index() + - removed_column_count( + requirement_map_right, + column.index(), + ), + )))) }) - .collect::>() - }) - .unwrap_or_default(), - ); - hj_left_requirements + .unwrap() + .data, + ) + }) + .collect() } -fn collect_hj_right_requirements( - all_requirements: &IndexSet, - join_projection: &[usize], - join_left_input_size: usize, - join_right_schema: SchemaRef, - on: &[(PhysicalExprRef, PhysicalExprRef)], +/// 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 { - let mut hj_right_requirements = all_requirements + required_columns .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::>(), - ); + .filter(|col| col.index() >= left_size) + .map(|col| Column::new(col.name(), col.index() - left_size)) + .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::>() +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()), + ) }) - .unwrap_or_default(), - ); - hj_right_requirements + .collect() + } } -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::>(), - ); +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(), + ), + ) + })); - 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::>() + 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) + } }) - .unwrap_or_default(), - ); - hj_right_requirements + .collect(); + node.schema_mapping.extend(self_node_map); + } } -#[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, +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() } -/// 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_mut(&mut |expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - if sync_with_child { - state = RewriteState::RewrittenValid; - // Update the index of `column`: - Ok(Transformed::yes(projected_exprs[column.index()].0.clone())) - } else { - // default to invalid, in case we can't find the relevant column - state = RewriteState::RewrittenInvalid; - // Determine how to update `column` to accommodate `projected_exprs` - projected_exprs - .iter() - .enumerate() - .find_map(|(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - column.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)) +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() } -/// Rewrites the expressions with new index values. -fn update_expressions( - exprs: &[Arc], - mapping: &IndexMap, -) -> Vec> { +/// 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| update_column_index(expr, mapping)) - .collect::>() -} - -/// Rewrites the sort expressions with new index values. -fn update_sort_expressions( - sort_exprs: &[PhysicalSortExpr], - mapping: &IndexMap, -) -> LexOrdering { - sort_exprs - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, mapping), - options: sort_expr.options, + .map(|(expr, alias)| { + let new_expr = expr + .clone() + .transform_up_mut(&mut |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::>() -} - -/// Updates the expressions subject to hashing of the `Partitioning` according to -/// the mapping. If it is not a hash partitioning, they remains as they are. -fn update_partitioning_expressions( - partitioning: &Partitioning, - mapping: &IndexMap, -) -> Partitioning { - if let Partitioning::Hash(exprs, size) = partitioning { - let updated_exprs = update_expressions(exprs, mapping); - Partitioning::Hash(updated_exprs, *size) - } else { - partitioning.clone() - } + .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| { - let new_exprs = update_expressions(&window_expr.expressions(), mapping); - window_expr.clone().with_new_expressions(new_exprs) +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::>>() -} + .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)], + }; -/// 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().with_new_expressions( - aggr_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(), - ) + 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::>>() + .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)) } -/// 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 { - join_on - .iter() - .map(|(left, right)| { - ( - update_column_index(left, left_mapping), - update_column_index(right, right_mapping), - ) +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() + .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 = hj_right_requirements.clone(); + + Ok((new_left_node, right_node)) } -/// Given mapping representing the initial and new index values, -/// it updates the indices of columns in the [`PhysicalExpr`]. -fn update_column_index( - expr: &Arc, - mapping: &IndexMap, -) -> Arc { - let mut state = RewriteState::Unchanged; - let new_expr = expr - .clone() - .transform_up_mut(&mut |expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - state = RewriteState::RewrittenValid; - // Update the index of `column`: - if let Some(updated) = mapping.get(column) { - Ok(Transformed::yes(Arc::new(updated.clone()) as _)) - } else { - Ok(Transformed::no(expr.clone())) - } +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) }) - .unwrap() - .data; - new_expr + .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 = hj_left_requirements.clone(); + + Ok((left_node, new_right_node)) } -/// 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_mut(&mut |expr: Arc| { - if left_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - left_state = RewriteState::RewrittenValid; - Ok(Transformed::yes(Arc::new(Column::new( - column.name(), - column.index() - - removed_column_count( - requirement_map_left, - column.index(), - ), - )))) - }) - .unwrap() - .data, - right_col - .clone() - .transform_up_mut(&mut |expr: Arc| { - if right_state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - right_state = RewriteState::RewrittenValid; - Ok(Transformed::yes(Arc::new(Column::new( - column.name(), - column.index() - - removed_column_count( - requirement_map_right, - column.index(), - ), - )))) - }) - .unwrap() - .data, - ) - }) - .collect() +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) } -/// Updates the [`JoinFilter`] according to the new indices of columns. -fn update_non_equivalence_conditions( +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. +/// +/// This function updates the column indices in a filter's predicate based on a provided mapping. +/// It creates a new `FilterExec` with the updated predicate. +/// +/// # Arguments +/// * `predicate` - The predicate expression of the filter. +/// * `input_plan` - The input execution plan on which the filter is applied. +/// * `mapping` - An IndexMap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `FilterExec` wrapped in an `Arc`. +fn rewrite_filter( + predicate: &Arc, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + FilterExec::try_new(update_column_index(predicate, mapping), input_plan) + .map(|plan| Arc::new(plan) as _) +} + +fn rewrite_hj_filter( filter: Option<&JoinFilter>, - requirement_map_left: &ColumnRequirements, - requirement_map_right: &ColumnRequirements, + left_mapping: &IndexMap, + right_mapping: &IndexMap, ) -> Option { filter.map(|filter| { JoinFilter::new( @@ -4751,17 +4700,26 @@ fn update_non_equivalence_conditions( 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, - }, + .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(), @@ -4769,821 +4727,856 @@ fn update_non_equivalence_conditions( }) } -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() - } +/// Rewrites a projection execution plan with updated column indices. +/// +/// This function updates the column indices in a projection based on a provided mapping. +/// It creates a new `ProjectionExec` with the updated expressions. +/// +/// # Arguments +/// * `projection` - The original projection execution plan. +/// * `input_plan` - The input execution plan on which the projection is applied. +/// * `mapping` - An IndexMap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `ProjectionExec` wrapped in an `Arc`. +/// +fn rewrite_projection( + projection: &ProjectionExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + ProjectionExec::try_new( + projection + .expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.clone())) + .collect::>(), + input_plan, + ) + .map(|plan| Arc::new(plan) as _) } -fn 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(), - ), - ) - })); +/// Rewrites a repartition execution plan with updated column indices. +/// +/// Updates the partitioning expressions in a repartition plan based on the provided column index mappings. +/// Supports updating the `Partitioning::Hash` variant of partitioning. +/// +/// # Arguments +/// * `partitioning` - The original partitioning strategy. +/// * `input_plan` - The input execution plan on which repartitioning is applied. +/// * `mapping` - An IndexMap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `RepartitionExec` wrapped in an `Arc`. +fn rewrite_repartition( + partitioning: &Partitioning, + input_plan: Arc, + mapping: &IndexMap, +) -> Result> { + let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { + let new_exprs = update_expressions(exprs, mapping); + Partitioning::Hash(new_exprs, *size) } 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(), - ), - ) - })); + partitioning.clone() + }; + RepartitionExec::try_new(input_plan, new_partitioning).map(|plan| Arc::new(plan) as _) +} - 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); - } +/// Rewrites a sort execution plan with updated column indices. +/// +/// This function updates the column indices in a sort's expressions based on a provided mapping. +/// It creates a new `SortExec` with the updated expressions. +/// +/// # Arguments +/// * `sort` - The original sort execution plan. +/// * `input_plan` - The input execution plan on which sorting is applied. +/// * `mapping` - An IndexMap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `SortExec` wrapped in an `Arc`. +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 _) } -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() +/// Rewrites a sort preserving merge execution plan with updated column indices. +/// +/// Updates the sort expressions in a sort preserving merge plan based on the provided column index mappings. +/// +/// # Arguments +/// * `sort` - The original `SortPreservingMergeExec` plan. +/// * `input_plan` - The input execution plan to which the sort preserving merge is applied. +/// * `mapping` - An IndexMap with old and new column index mappings. +/// +/// # Returns +/// A `Result` containing the new `SortPreservingMergeExec` wrapped in an `Arc`. +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 update_right_requirements( - required_columns: IndexSet, +fn rewrite_nested_loop_join( + nlj: &NestedLoopJoinExec, + left_input_plan: Arc, + right_input_plan: Arc, + left_mapping: &IndexMap, + right_mapping: &IndexMap, left_size: usize, -) -> IndexSet { - required_columns - .into_iter() - .map(|col| Column::new(col.name(), col.index() + left_size)) - .collect() +) -> 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 _) } -/// 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_mut(&mut |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::>>() +/// 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. +/// +/// # Arguments +/// * `smj`: The original SortMergeJoinExec to be rewritten. +/// * `left_input_plan`: The updated execution plan for the left input. +/// * `right_input_plan`: The updated execution plan for the right input. +/// * `left_mapping`: Column mapping for the left input. +/// * `right_mapping`: Column mapping for the right input. +/// * `left_size`: The size of the left input, necessary for index calculations. +/// +/// # Returns +/// A Result containing the rewritten execution plan as an Arc, or an error on failure. +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 _) } -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)], - }; +/// 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. +/// +/// # Arguments +/// * `shj`: The SymmetricHashJoinExec to be modified. +/// * `left_input_plan`: New execution plan for the left side. +/// * `right_input_plan`: New execution plan for the right side. +/// * `left_mapping`: Mapping for updating left side columns. +/// * `right_mapping`: Mapping for updating right side columns. +/// * `left_size`: Size of the left input for index adjustments. +/// +/// # Returns +/// A Result containing the updated execution plan within an Arc, or an error if the operation fails. +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(), + ) + }); - Ok((new_left_node, new_right_node)) + let new_left_sort_exprs = shj.left_sort_exprs().map(|exprs| { + exprs + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, left_mapping), + options: sort_expr.options, + }) + .collect() + }); + let new_right_sort_exprs = shj.left_sort_exprs().map(|exprs| { + exprs + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: update_column_index(&sort_expr.expr, right_mapping), + 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 _) } -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) +/// 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. +/// +/// # Arguments +/// * `agg`: The original AggregateExec to be rewritten. +/// * `input_plan`: The new execution plan to use as input. +/// * `mapping`: A mapping from old to new columns. +/// +/// # Returns +/// A Result that either contains an Option with the new execution plan wrapped in an Arc, or None if no rewriting is possible, along with an error on failure. +fn rewrite_aggregate( + agg: &AggregateExec, + input_plan: Arc, + mapping: &IndexMap, +) -> Result>> { + let new_group_by = PhysicalGroupBy::new( + agg.group_expr() + .expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .collect(), + agg.group_expr() + .null_expr() + .iter() + .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .collect(), + agg.group_expr().groups().to_vec(), + ); + let new_agg_expr = + if let Some(new_agg_expr) = update_aggregate_exprs(agg.aggr_expr(), mapping) { + new_agg_expr + } else { + return Ok(None); + }; + let new_filter = agg + .filter_expr() + .iter() + .map(|opt_expr| { + opt_expr + .clone() + .map(|expr| update_column_index(&expr, mapping)) }) - .collect::>(); - 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)], - }; + .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 _)) +} - let mut right_node = children.swap_remove(0); - right_node.required_columns = hj_right_requirements.clone(); +/// 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. +/// +/// # Arguments +/// * `w_agg`: The WindowAggExec to be updated. +/// * `input_plan`: The new execution plan to be used. +/// * `mapping`: Column mapping for updating window expressions. +/// +/// # Returns +/// A Result containing either an Option with the updated execution plan in an Arc, or None if rewriting isn't feasible, and an error on failure. +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 new_partition_keys = update_expressions(&w_agg.partition_keys, mapping); + WindowAggExec::try_new(new_window, input_plan, new_partition_keys) + .map(|plan| Some(Arc::new(plan) as _)) +} - Ok((new_left_node, right_node)) +/// 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. +/// +/// # Arguments +/// * `bw_agg`: The original BoundedWindowAggExec to be rewritten. +/// * `input_plan`: The new execution plan to use. +/// * `mapping`: Mapping for updating window expressions. +/// +/// # Returns +/// A Result containing an Option with the new execution plan wrapped in an Arc, or None if the rewrite is not possible, and an error on failure. +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 new_partition_keys = update_expressions(&bw_agg.partition_keys, mapping); + BoundedWindowAggExec::try_new( + new_window, + input_plan, + new_partition_keys, + bw_agg.input_order_mode.clone(), + ) + .map(|plan| Some(Arc::new(plan) as _)) } -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 +fn split_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() - .map(|req| { - let name = req.name().to_owned(); - (Arc::new(req) as Arc, name) + .enumerate() + .filter_map(|(idx, csv_idx)| { + if used_indices.contains(&csv_idx) { + None + } else { + Some(idx) + } }) .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 = hj_left_requirements.clone(); + (used_indices, unused_indices) +} - Ok((left_node, new_right_node)) +/// 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_schema_agnostic(plan: &Arc) -> bool { + let plan_any = plan.as_any(); + plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() } -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) +/// 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() } -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() - }) +/// 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() } -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() - }) +/// 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()) } -/// Rewrites a filter execution plan with updated column indices. -/// -/// This function updates the column indices in a filter's predicate based on a provided mapping. -/// It creates a new `FilterExec` with the updated predicate. -/// -/// # Arguments -/// * `predicate` - The predicate expression of the filter. -/// * `input_plan` - The input execution plan on which the filter is applied. -/// * `mapping` - A IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `FilterExec` wrapped in an `Arc`. -fn rewrite_filter( - predicate: &Arc, - input_plan: Arc, - mapping: &IndexMap, -) -> Result> { - FilterExec::try_new(update_column_index(predicate, mapping), input_plan) - .map(|plan| Arc::new(plan) as _) +/// Tries to rewrite the [`WindowExpr`] with the existing expressions to keep on optimization. +fn is_window_expr_rewritable(window_expr: &[Arc<(dyn WindowExpr)>]) -> bool { + window_expr.iter().all(|expr| expr.expressions().is_empty()) } -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(), - ) - }) +/// 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 } -/// Rewrites a projection execution plan with updated column indices. +/// Analyzes the column requirements for join operations between left and right children plans. /// -/// This function updates the column indices in a projection based on a provided mapping. -/// It creates a new `ProjectionExec` with the updated expressions. +/// 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. +/// Each column in the requirement maps corresponds to its own table schema index, not to the join output schema. /// /// # Arguments -/// * `projection` - The original projection execution plan. -/// * `input_plan` - The input execution plan on which the projection is applied. -/// * `mapping` - A IndexMap with old and new column index mappings. +/// * `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 `Result` containing the new `ProjectionExec` wrapped in an `Arc`. +/// 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 rewrite_projection( +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)) +} + +// 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, - input_plan: Arc, - mapping: &IndexMap, -) -> Result> { - ProjectionExec::try_new( - projection - .expr() + 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(&mut |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() - .map(|(expr, alias)| (update_column_index(expr, mapping), alias.clone())) - .collect::>(), - input_plan, - ) - .map(|plan| Arc::new(plan) as _) + .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 + }) + } } -/// Rewrites a repartition execution plan with updated column indices. -/// -/// Updates the partitioning expressions in a repartition plan based on the provided column index mappings. -/// Supports updating the `Partitioning::Hash` variant of partitioning. -/// -/// # Arguments -/// * `partitioning` - The original partitioning strategy. -/// * `input_plan` - The input execution plan on which repartitioning is applied. -/// * `mapping` - A IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `RepartitionExec` wrapped in an `Arc`. -fn rewrite_repartition( - partitioning: &Partitioning, - input_plan: Arc, - mapping: &IndexMap, -) -> Result> { - let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { - let new_exprs = update_expressions(exprs, mapping); - Partitioning::Hash(new_exprs, *size) - } else { - partitioning.clone() - }; - RepartitionExec::try_new(input_plan, new_partitioning).map(|plan| Arc::new(plan) as _) +fn window_agg_required( + original_schema_len: usize, + requirements: &ColumnRequirements, +) -> bool { + requirements + .iter() + .filter(|(column, _used)| column.index() >= original_schema_len) + .any(|(_column, used)| *used) } -/// Rewrites a sort execution plan with updated column indices. -/// -/// This function updates the column indices in a sort's expressions based on a provided mapping. -/// It creates a new `SortExec` with the updated expressions. -/// -/// # Arguments -/// * `sort` - The original sort execution plan. -/// * `input_plan` - The input execution plan on which sorting is applied. -/// * `mapping` - A IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `SortExec` wrapped in an `Arc`. -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 _) +/// Updates a source provider's projected columns according to the given +/// projection operator's expressions. To use this function safely, one must +/// ensure that all expressions are `Column` expressions without aliases. +fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec { + projection.iter().map(|col| source[col.index()]).collect() } -/// Rewrites a sort preserving merge execution plan with updated column indices. -/// -/// Updates the sort expressions in a sort preserving merge plan based on the provided column index mappings. -/// -/// # Arguments -/// * `sort` - The original `SortPreservingMergeExec` plan. -/// * `input_plan` - The input execution plan to which the sort preserving merge is applied. -/// * `mapping` - A IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `SortPreservingMergeExec` wrapped in an `Arc`. -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 _) +/// 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 } -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 _) +/// 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() } -/// 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. -/// -/// # Arguments -/// * `smj`: The original SortMergeJoinExec to be rewritten. -/// * `left_input_plan`: The updated execution plan for the left input. -/// * `right_input_plan`: The updated execution plan for the right input. -/// * `left_mapping`: Column mapping for the left input. -/// * `right_mapping`: Column mapping for the right input. -/// * `left_size`: The size of the left input, necessary for index calculations. -/// -/// # Returns -/// A Result containing the rewritten execution plan as an Arc, or an error on failure. -fn rewrite_sort_merge_join( - smj: &SortMergeJoinExec, - left_input_plan: Arc, - right_input_plan: Arc, - left_mapping: &IndexMap, - right_mapping: &IndexMap, +fn extend_left_mapping_with_right( + mut left_schema_mapping: IndexMap, + right_child_plan: &Arc, 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 _) + 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 } -/// 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. +/// Calculates the count of removed (unused) columns that precede a given column index. /// /// # Arguments -/// * `shj`: The SymmetricHashJoinExec to be modified. -/// * `left_input_plan`: New execution plan for the left side. -/// * `right_input_plan`: New execution plan for the right side. -/// * `left_mapping`: Mapping for updating left side columns. -/// * `right_mapping`: Mapping for updating right side columns. -/// * `left_size`: Size of the left input for index adjustments. +/// * `requirement_map`: Reference to a ColumnRequirements map. +/// * `column_index`: The index of the column in question. /// /// # Returns -/// A Result containing the updated execution plan within an Arc, or an error if the operation fails. -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(), +/// The number of removed columns before the 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) + } + }, ) - }); - - let new_left_sort_exprs = shj.left_sort_exprs().map(|exprs| { - exprs - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, left_mapping), - options: sort_expr.options, - }) - .collect() - }); - let new_right_sort_exprs = shj.left_sort_exprs().map(|exprs| { - exprs - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, right_mapping), - 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 _) + { + if unused_col.index() < column_index { + left_skipped_columns += 1; + } + } + left_skipped_columns } -/// Rewrites an aggregate execution plan. +/// Maps the indices of required columns in a parent projection node to the corresponding indices in its child. /// -/// This function updates an AggregateExec plan using a new input plan and column mappings. It adjusts group-by expressions, aggregate expressions, and filters. +/// # Example +/// Projection is required to have columns at "@0:a - @1:b - @2:c" /// -/// # Arguments -/// * `agg`: The original AggregateExec to be rewritten. -/// * `input_plan`: The new execution plan to use as input. -/// * `mapping`: A mapping from old to new columns. +/// Projection does "a@2 as a, b@0 as b, c@1 as c" /// -/// # Returns -/// A Result that either contains an Option with the new execution plan wrapped in an Arc, or None if no rewriting is possible, along with an error on failure. -fn rewrite_aggregate( - agg: &AggregateExec, - input_plan: Arc, - mapping: &IndexMap, -) -> Result>> { - let new_group_by = PhysicalGroupBy::new( - agg.group_expr() - .expr() - .iter() - .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) - .collect(), - agg.group_expr() - .null_expr() - .iter() - .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) - .collect(), - agg.group_expr().groups().to_vec(), - ); - let new_agg_expr = - if let Some(new_agg_expr) = update_aggregate_exprs(agg.aggr_expr(), mapping) { - new_agg_expr - } else { - return Ok(None); - }; - let new_filter = agg - .filter_expr() +/// 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(|opt_expr| { - opt_expr - .clone() - .map(|expr| update_column_index(&expr, mapping)) - }) - .collect(); - AggregateExec::try_new( - *agg.mode(), - new_group_by, - new_agg_expr, - new_filter, - input_plan, - agg.input_schema(), - ) - .map(|plan| Some(Arc::new(plan.with_limit(agg.limit())) as _)) + .map(|column| projection_columns[column.index()].clone()) + .collect::>() } -/// 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. -/// -/// # Arguments -/// * `w_agg`: The WindowAggExec to be updated. -/// * `input_plan`: The new execution plan to be used. -/// * `mapping`: Column mapping for updating window expressions. -/// -/// # Returns -/// A Result containing either an Option with the updated execution plan in an Arc, or None if rewriting isn't feasible, and an error on failure. -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 new_partition_keys = update_expressions(&w_agg.partition_keys, mapping); - WindowAggExec::try_new(new_window, input_plan, new_partition_keys) - .map(|plan| Some(Arc::new(plan) as _)) +/// 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() } -/// 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. -/// -/// # Arguments -/// * `bw_agg`: The original BoundedWindowAggExec to be rewritten. -/// * `input_plan`: The new execution plan to use. -/// * `mapping`: Mapping for updating window expressions. -/// -/// # Returns -/// A Result containing an Option with the new execution plan wrapped in an Arc, or None if the rewrite is not possible, and an error on failure. -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 +/// 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 { - return Ok(None); - }; - let new_partition_keys = update_expressions(&bw_agg.partition_keys, mapping); - BoundedWindowAggExec::try_new( - new_window, - input_plan, - new_partition_keys, - bw_agg.input_order_mode.clone(), - ) - .map(|plan| Some(Arc::new(plan) as _)) + unused.insert(col); + } + } + (required, unused) } #[cfg(test)] From 26cd56cf6e87895d91b13334eb92322633dbee78 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 19 Apr 2024 18:03:01 +0300 Subject: [PATCH 63/85] Self review part 2 --- .../optimize_projections.rs | 1828 ++++++----------- 1 file changed, 665 insertions(+), 1163 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index bcd23878385b..294fd4446175 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -196,14 +196,19 @@ impl ProjectionOptimizer { return match self.try_embed_to_hash_join()? { join if join.transformed => { // Re-run on the new HashJoin node - return join.data.adjust_node_with_requirements(); + 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) } - projection => Ok(projection.data), }; } // These kind of plans can swap the order with projections without any further modification. - if is_schema_agnostic(projection_input) { + if is_plan_schema_agnostic(projection_input) { self = match self.try_swap_trivial()? { swapped if swapped.transformed => { return Ok(swapped.data); @@ -368,6 +373,7 @@ impl ProjectionOptimizer { /// 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)); }; @@ -377,26 +383,24 @@ impl ProjectionOptimizer { }; // Collect all column indices from the given projection expressions. - let mut projection_index = + let mut projection_indices = collect_column_indices_in_proj_exprs(projection.expr()); - projection_index.sort_by_key(|i| *i); + 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_index.is_empty() - || (projection_index.len() == projection_index.last().unwrap() + 1 - && projection_index.len() == hash_join.schema().fields().len()) + if projection_indices.is_empty() + || (projection_indices.len() == projection_indices.last().unwrap() + 1 + && projection_indices.len() == hash_join.schema().fields().len()) { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(Transformed::no(self)); } + let new_hash_join = - Arc::new(hash_join.with_projection(Some(projection_index.clone()))?) + Arc::new(hash_join.with_projection(Some(projection_indices.clone()))?) as Arc; - // Build projection expressions for update_expr. Zip the projection_index with the new_hash_join output schema fields. - let embed_project_exprs = projection_index + let builtin_projection_exprs = projection_indices .iter() .zip(new_hash_join.schema().fields()) .map(|(index, field)| { @@ -411,10 +415,8 @@ impl ProjectionOptimizer { 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, embed_project_exprs.as_slice(), false)? + update_expr_with_projection(expr, &builtin_projection_exprs, false)? else { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan); return Ok(Transformed::no(self)); }; new_projection_exprs.push((expr, alias.clone())); @@ -434,10 +436,9 @@ impl ProjectionOptimizer { children_nodes: self.children_nodes.swap_remove(0).children_nodes, })) } else { - let required_columns = collect_columns_in_plan_schema(&new_hash_join); let new_join_node = Self { plan: new_hash_join, - required_columns, + required_columns: IndexSet::new(), schema_mapping: IndexMap::new(), children_nodes: self.children_nodes.swap_remove(0).children_nodes, }; @@ -489,19 +490,21 @@ impl ProjectionOptimizer { 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 mut file_scan = csv.base_config().clone(); - let new_projections = new_projections_for_columns( - &projection_columns, - &file_scan - .projection - .unwrap_or((0..csv.schema().fields().len()).collect()), - ); - + 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 { @@ -518,14 +521,13 @@ impl ProjectionOptimizer { children_nodes: vec![], }) } else { - let mut file_scan = csv.base_config().clone(); 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) = - split_column_indices(file_scan_projection, required_columns); + 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 { @@ -562,20 +564,10 @@ impl ProjectionOptimizer { let plan = self.plan.clone(); // These plans preserve the input schema, and do not add new requirements. - if let Some(coal_batches) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_coalesce_batches(coal_batches)?; - } else if plan - .as_any() - .downcast_ref::() - .is_some() - { - self = self.try_insert_below_coalesce_partitions()?; - } else if let Some(global_limit) = plan.as_any().downcast_ref::() - { - self = self.try_insert_below_global_limit(global_limit)?; - } else if let Some(local_limit) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_local_limit(local_limit)?; + if is_plan_schema_agnostic(&plan) { + self = self.try_insert_below_schema_agnostic()?; } + // ------------------------------------------------------------------------ // These plans also preserve the input schema, but may extend requirements. else if let Some(filter) = plan.as_any().downcast_ref::() { self = self.try_insert_below_filter(filter)?; @@ -589,16 +581,19 @@ impl ProjectionOptimizer { { self = self.try_insert_below_sort_preserving_merge(sort_merge)?; } + // ------------------------------------------------------------------------ // Preserves schema and do not change requirements, but have multi-child. else if plan.as_any().downcast_ref::().is_some() { self = self.try_insert_below_union()?; } else if plan.as_any().downcast_ref::().is_some() { self = self.try_insert_below_interleave()?; } + // ------------------------------------------------------------------------ // Concatenates schemas and do not change requirements. else if let Some(cj) = plan.as_any().downcast_ref::() { self = self.try_insert_below_cross_join(cj)? } + // ------------------------------------------------------------------------ // Joins and aggregations require special attention. else if let Some(hj) = plan.as_any().downcast_ref::() { self = self.try_insert_below_hash_join(hj)? @@ -639,133 +634,16 @@ impl ProjectionOptimizer { Ok(self) } - /// Attempts to insert a projection node below a `CoalesceBatchesExec` node. - /// - /// This method checks if there is any redundancy in the execution plan based on the requirements - /// of the `CoalesceBatchesExec` node. If all columns are required, it updates the required columns of the child node. - /// Otherwise, it inserts a new projection to optimize the plan. - /// - /// # Arguments - /// * `coal_batches`: Reference to a `CoalesceBatchesExec` object. - /// - /// # Returns - /// * `Result`: The modified `ProjectionOptimizer` after potentially inserting a projection. - fn try_insert_below_coalesce_batches( - mut self, - coal_batches: &CoalesceBatchesExec, - ) -> Result { - // CoalesceBatchesExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = analyze_requirements(&self); - if all_columns_required(&requirement_map) { - self.children_nodes[0].required_columns = - mem::take(&mut self.required_columns); - } else { - let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - - let plan = Arc::new(CoalesceBatchesExec::new( - new_child.plan.clone(), - coal_batches.target_batch_size(), - )) as _; - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - - /// Attempts to insert a projection node below a `CoalescePartitionsExec` node. - /// - /// Similar to `try_insert_below_coalesce_batches`, this method analyzes the requirements of the - /// `CoalescePartitionsExec` node and modifies the execution plan accordingly. - /// - /// # Returns - /// * `Result`: The potentially modified `ProjectionOptimizer`. - fn try_insert_below_coalesce_partitions(mut self) -> Result { - // CoalescePartitionsExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = analyze_requirements(&self); - if all_columns_required(&requirement_map) { - self.children_nodes[0].required_columns = - mem::take(&mut self.required_columns); - } else { - let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - let plan = Arc::new(CoalescePartitionsExec::new(new_child.plan.clone())) as _; - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - - /// Attempts to insert a projection node below a `GlobalLimitExec` node. - /// - /// Analyzes the requirements imposed by `GlobalLimitExec` and optimizes the plan by potentially - /// inserting a projection node to reduce the number of columns processed. - /// - /// # Arguments - /// * `global_limit`: Reference to a `GlobalLimitExec` object. - /// - /// # Returns - /// * `Result`: The modified `ProjectionOptimizer`. - fn try_insert_below_global_limit( - mut self, - global_limit: &GlobalLimitExec, - ) -> Result { - // GlobalLimitExec does not change requirements. We can directly check whether there is a redundancy. + 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) { - // if all_columns_required(&requirement_map) { self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - let plan = Arc::new(GlobalLimitExec::new( - new_child.plan.clone(), - global_limit.skip(), - global_limit.fetch(), - )) as _; - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - /// Attempts to insert a projection node below a `LocalLimitExec` node. - /// - /// Optimizes the plan considering the requirements of `LocalLimitExec`, potentially inserting a projection node. - /// - /// # Arguments - /// * `limit`: Reference to a `LocalLimitExec` object. - /// - /// # Returns - /// * `Result`: The updated `ProjectionOptimizer`. - fn try_insert_below_local_limit( - mut self, - local_limit: &LocalLimitExec, - ) -> Result { - // LocalLimitExec does not change requirements. We can directly check whether there is a redundancy. - 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 (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - let plan = Arc::new(LocalLimitExec::new( - new_child.plan.clone(), - local_limit.fetch(), - )) as _; + let plan = plan.with_new_children(vec![new_child.plan.clone()])?; self = ProjectionOptimizer { plan, @@ -777,16 +655,8 @@ impl ProjectionOptimizer { Ok(self) } - /// Attempts to insert a projection node below a `FilterExec` node. - /// - /// Extends the required columns with those in the filter's predicate and optimizes the plan, potentially inserting - /// a projection node. - /// - /// # Arguments - /// * `filter`: Reference to a `FilterExec` object. - /// - /// # Returns - /// * `Result`: The optimized `ProjectionOptimizer`. + /// Attempts to insert a projection node below a `FilterExec` node. Extends the required columns + /// with those in the filter's predicate and optimizes the plan, potentially inserting a projection node. fn try_insert_below_filter( mut self, filter: &FilterExec, @@ -817,16 +687,9 @@ impl ProjectionOptimizer { Ok(self) } - /// Attempts to insert a projection node below a `RepartitionExec` node. - /// - /// If `RepartitionExec` involves a hash repartition, it extends the requirements with the columns in the hashed expressions. + /// Attempts to insert a projection node below a `RepartitionExec` node. If `RepartitionExec` involves + /// a hash repartition, it extends the requirements with the columns in the hashed expressions. /// The method then optimizes the execution plan accordingly. - /// - /// # Arguments - /// * `repartition`: Reference to a `RepartitionExec` object. - /// - /// # Returns - /// * `Result`: The potentially updated `ProjectionOptimizer`. fn try_insert_below_repartition( mut self, repartition: &RepartitionExec, @@ -844,7 +707,7 @@ impl ProjectionOptimizer { mem::take(&mut self.required_columns); } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - // Rewrite the hashed expressions if there is any with possibly updated column indices. + // Rewrite the expression if there is any with possibly updated column indices. let new_partitioning = update_partitioning_expressions( repartition.partitioning(), &schema_mapping, @@ -864,16 +727,8 @@ impl ProjectionOptimizer { Ok(self) } - /// Attempts to insert a projection node below a `SortExec` node. - /// - /// Extends the requirements with columns involved in the sort expressions and optimizes the execution plan, - /// potentially inserting a projection node. - /// - /// # Arguments - /// * `sort`: Reference to a `SortExec` object. - /// - /// # Returns - /// * `Result`: The optimized `ProjectionOptimizer`. + /// Attempts to insert a projection node below a `SortExec` node. Extends the requirements with columns + /// involved in the sort expressions and optimizes the execution plan, potentially inserting a projection node. fn try_insert_below_sort(mut self, sort: &SortExec) -> Result { // SortExec extends the requirements with the columns in its sort expressions. self.required_columns.extend( @@ -906,16 +761,8 @@ impl ProjectionOptimizer { Ok(self) } - /// Attempts to insert a projection node below a `SortPreservingMergeExec` node. - /// - /// Similar to `try_insert_below_sort`, it extends the requirements with columns in the sort expressions and - /// optimizes the plan accordingly. - /// - /// # Arguments - /// * `sortp_merge`: Reference to a `SortPreservingMergeExec` object. - /// - /// # Returns - /// * `Result`: The modified `ProjectionOptimizer`. + /// Attempts to insert a projection node below a `SortPreservingMergeExec` node. Similar to `try_insert_below_sort`, + /// it extends the requirements with columns in the sort expressions and optimizes the plan accordingly. fn try_insert_below_sort_preserving_merge( mut self, sort_merge: &SortPreservingMergeExec, @@ -959,9 +806,6 @@ impl ProjectionOptimizer { /// requirement map, the method updates the required columns for all child nodes accordingly. If not all /// columns are required, it inserts new projection nodes to optimize the plan, leading to a more efficient /// execution by reducing unnecessary data processing. - /// - /// # Returns - /// * `Result`: The optimized `ProjectionOptimizer` after potentially inserting projection nodes. fn try_insert_below_union(mut self) -> Result { // UnionExec does not change requirements. We can directly check whether there is a redundancy. let requirement_map = analyze_requirements(&self); @@ -993,9 +837,6 @@ impl ProjectionOptimizer { /// modifies the execution plan accordingly. If all columns are required, it updates the required columns for /// each child node. Otherwise, it inserts new projection nodes for optimization. This process can lead to a /// more efficient execution by minimizing the data processed in the context of interleaved execution. - /// - /// # Returns - /// * `Result`: The potentially modified `ProjectionOptimizer` after the optimization process. fn try_insert_below_interleave(mut self) -> Result { let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { @@ -1027,12 +868,6 @@ impl ProjectionOptimizer { /// 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. - /// - /// # Arguments - /// * `cj`: Reference to the `CrossJoinExec` node in the plan. - /// - /// # Returns - /// * `Result`: The updated `ProjectionOptimizer` after potentially inserting projection nodes. fn try_insert_below_cross_join( mut self, cj: &CrossJoinExec, @@ -1073,17 +908,16 @@ impl ProjectionOptimizer { (false, true) => { let required_columns = mem::take(&mut self.required_columns); let mut right_child = self.children_nodes.swap_remove(1); - - 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 (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, @@ -1102,15 +936,14 @@ impl ProjectionOptimizer { (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); - left_child.required_columns = - collect_left_used_columns(required_columns, left_size); - let plan = Arc::new(CrossJoinExec::new( left_child.plan.clone(), new_right_child.plan.clone(), @@ -1143,24 +976,17 @@ impl ProjectionOptimizer { /// and reorganizing the required columns as needed. This function supports various /// join types, including Inner, Left, Right, Full, LeftAnti, LeftSemi, RightAnti, /// and RightSemi joins. - /// - /// # Arguments - /// - /// * `hj`: Reference to a HashJoinExec node representing the join operation in the query plan. - /// - /// # Returns - /// - /// Returns `Result`, which is an updated ProjectionOptimizer - /// instance after potentially adding projections below the HashJoinExec node. - /// On success, it contains the modified ProjectionOptimizer with new projections - /// and updated plan. On failure, it returns an error indicating the issue encountered - /// during the operation. fn try_insert_below_hash_join( mut self, hj: &HashJoinExec, ) -> Result { match hj.join_type() { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + 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 @@ -1214,6 +1040,7 @@ impl ProjectionOptimizer { 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(), @@ -1337,26 +1164,21 @@ impl ProjectionOptimizer { } } } - JoinType::LeftSemi | JoinType::LeftAnti => { - let join_left_input_size = hj.left().schema().fields().len(); + JoinType::RightSemi | JoinType::RightAnti => { let join_projection = hj .projection .clone() - .unwrap_or((0..hj.left().schema().fields().len()).collect()); + .unwrap_or((0..hj.right().schema().fields().len()).collect()); - let hj_left_requirements = collect_hj_left_requirements( - &self.required_columns, - &join_projection, - join_left_input_size, + let hj_left_requirements = collect_right_hj_left_requirements( hj.left().schema(), hj.on(), hj.filter(), ); - let hj_right_requirements = collect_hj_right_requirements( + let hj_right_requirements = collect_right_hj_right_requirements( &self.required_columns, &join_projection, - join_left_input_size, hj.right().schema(), hj.on(), hj.filter(), @@ -1392,13 +1214,9 @@ impl ProjectionOptimizer { let new_filter = rewrite_hj_filter(hj.filter(), &left_mapping, &right_mapping); - let new_projection = update_hj_projection( + let new_projection = update_hj_projection_right( hj.projection.clone(), - hj.left().schema(), - hj_left_requirements, - left_mapping, right_mapping, - join_left_input_size, ); let new_hash_join = HashJoinExec::try_new( @@ -1435,13 +1253,9 @@ impl ProjectionOptimizer { 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( + let new_projection = update_hj_projection_right( hj.projection.clone(), - hj.left().schema(), - hj_left_requirements, left_mapping, - right_mapping, - join_left_input_size, ); let new_hash_join = HashJoinExec::try_new( @@ -1479,13 +1293,9 @@ impl ProjectionOptimizer { 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( + let new_projection = update_hj_projection_right( hj.projection.clone(), - hj.left().schema(), - hj_left_requirements, - left_mapping, right_mapping, - join_left_input_size, ); let new_hash_join = HashJoinExec::try_new( @@ -1515,210 +1325,58 @@ impl ProjectionOptimizer { } } } - 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) - }); + /// 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 (keep_left_same, keep_right_same) { + 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_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) - } - } - } - } - } - - 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_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( @@ -1907,6 +1565,15 @@ impl ProjectionOptimizer { 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, @@ -2174,6 +1841,15 @@ impl ProjectionOptimizer { 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, @@ -2471,7 +2147,7 @@ impl ProjectionOptimizer { .iter() .map(|req_col| req_col.index()) .collect::>(); - let unused_aggr_exprs = agg + let unused_aggr_expr_indices = agg .aggr_expr() .iter() .enumerate() @@ -2479,14 +2155,14 @@ impl ProjectionOptimizer { .map(|(idx, _expr)| idx) .collect::>(); - if !unused_aggr_exprs.is_empty() { + 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_exprs.contains(idx)) + .filter(|(idx, _expr)| !unused_aggr_expr_indices.contains(idx)) .map(|(_idx, expr)| expr.clone()) .collect(), agg.filter_expr().to_vec(), @@ -2510,11 +2186,22 @@ impl ProjectionOptimizer { self.plan = Arc::new(new_plan); self.required_columns = IndexSet::new(); } else { - match agg.mode() { + 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(); - let aggr_columns = agg - .aggr_expr() + agg.aggr_expr() .iter() .flat_map(|e| { e.state_fields().map(|field| { @@ -2528,57 +2215,25 @@ impl ProjectionOptimizer { }) }) .flatten() - .collect::>(); - let group_columns = agg - .group_expr() - .expr() - .iter() - .flat_map(|(expr, _name)| collect_columns(expr)) - .collect::>(); - let filter_columns = agg - .filter_expr() - .iter() - .filter_map(|expr| expr.as_ref().map(collect_columns)) - .flatten() - .collect::>(); - self.children_nodes[0].required_columns.extend( - aggr_columns - .into_iter() - .chain(group_columns) - .chain(filter_columns), - ) - } - _ => { - let aggr_columns = agg - .aggr_expr() - .iter() - .flat_map(|e| { - e.expressions() - .iter() - .flat_map(collect_columns) - .collect::>() - }) - .collect::>(); - let group_columns = agg - .group_expr() - .expr() - .iter() - .flat_map(|(expr, _name)| collect_columns(expr)) - .collect::>(); - let filter_columns = agg - .filter_expr() - .iter() - .filter_map(|expr| expr.as_ref().map(collect_columns)) - .flatten() - .collect::>(); - self.children_nodes[0].required_columns.extend( - aggr_columns - .into_iter() - .chain(group_columns) - .chain(filter_columns), - ); + .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) } @@ -2600,6 +2255,7 @@ impl ProjectionOptimizer { self.required_columns .extend(w_agg.partition_keys.iter().flat_map(collect_columns)); let requirement_map = analyze_requirements(&self); + if !all_columns_required(&requirement_map) { if window_agg_required( w_agg.input().schema().fields().len(), @@ -2640,7 +2296,8 @@ impl ProjectionOptimizer { .collect::>>() else { return Err(datafusion_common::DataFusionError::Internal( - "".to_string(), + format!("Window expression {:?} must implement with_new_expressions() API", w_agg + .window_expr()) )); }; @@ -2745,7 +2402,8 @@ impl ProjectionOptimizer { .collect::>>() else { return Err(datafusion_common::DataFusionError::Internal( - "".to_string(), + format!("Bounded window expression {:?} must implement with_new_expressions() API", bw_agg + .window_expr()) )); }; @@ -2814,13 +2472,12 @@ impl ProjectionOptimizer { self.plan.children()[0].clone(), )?) as _; + let new_requirements = collect_columns_in_plan_schema(&inserted_projection); 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, @@ -2837,8 +2494,8 @@ impl ProjectionOptimizer { // 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 projected_exprs = convert_projection_exprs(used_columns); let inserted_projections = self .plan .children() @@ -2851,15 +2508,15 @@ impl ProjectionOptimizer { }) .collect::>>()?; - let new_mapping = - calculate_column_mapping(&self.required_columns, &unused_columns); - 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) @@ -2874,25 +2531,65 @@ impl ProjectionOptimizer { Ok((inserted_projection_nodes, new_mapping)) } - /// Single child version of `insert_projection` for joins. - #[allow(clippy::type_complexity)] - fn insert_projection_below_multi_child( - mut self, + /// 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, - 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) = + ) -> 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 new_left_mapping = calculate_column_mapping(&used_columns, &unused_columns); - let projected_exprs = convert_projection_exprs(used_columns); + let projected_exprs = convert_projection_exprs(left_used_columns); let inserted_projection = Arc::new(ProjectionExec::try_new(projected_exprs, child_plan)?) as _; @@ -2904,11 +2601,8 @@ impl ProjectionOptimizer { children_nodes: vec![self.children_nodes.swap_remove(0)], }; - let (used_columns, unused_columns) = - partition_column_requirements(requirement_map_right); let child_plan = self.plan.children().remove(1); - let new_right_mapping = calculate_column_mapping(&used_columns, &unused_columns); - let projected_exprs = convert_projection_exprs(used_columns); + let projected_exprs = convert_projection_exprs(right_used_columns); let inserted_projection = Arc::new(ProjectionExec::try_new(projected_exprs, child_plan)?) as _; @@ -2919,12 +2613,7 @@ impl ProjectionOptimizer { schema_mapping: IndexMap::new(), children_nodes: vec![self.children_nodes.swap_remove(0)], }; - Ok(( - left_inserted_projection, - right_inserted_projection, - new_left_mapping, - new_right_mapping, - )) + Ok((left_inserted_projection, right_inserted_projection)) } /// Left child version of `insert_projection` for joins. @@ -2932,11 +2621,10 @@ impl ProjectionOptimizer { mut self, requirement_map_left: ColumnRequirements, ) -> Result<(Self, 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 child_plan = self.plan.children().remove(0); let (used_columns, unused_columns) = partition_column_requirements(requirement_map_left); - let child_plan = self.plan.children().remove(0); + let new_mapping = calculate_column_mapping(&used_columns, &unused_columns); let projected_exprs = convert_projection_exprs(used_columns); let inserted_projection = @@ -2949,6 +2637,7 @@ impl ProjectionOptimizer { schema_mapping: IndexMap::new(), children_nodes: vec![self.children_nodes.swap_remove(0)], }; + Ok((inserted_projection, new_mapping)) } @@ -2957,11 +2646,10 @@ impl ProjectionOptimizer { mut self, requirement_map_right: ColumnRequirements, ) -> Result<(Self, 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 child_plan = self.plan.children().remove(1); let (used_columns, unused_columns) = partition_column_requirements(requirement_map_right); - let child_plan = self.plan.children().remove(1); + let new_mapping = calculate_column_mapping(&used_columns, &unused_columns); let projected_exprs = convert_projection_exprs(used_columns); let inserted_projection = @@ -2977,55 +2665,8 @@ impl ProjectionOptimizer { Ok((inserted_projection, new_mapping)) } - /// Multi-child version of `insert_projection` for joins. - fn insert_multi_projections_below_join( - self, - left_size: usize, - requirement_map_left: ColumnRequirements, - requirement_map_right: ColumnRequirements, - ) -> Result<(Self, Self, IndexMap)> { - let original_right = self.children_nodes[1].plan.clone(); - let ( - new_left_child, - new_right_child, - mut left_schema_mapping, - right_schema_mapping, - ) = self.insert_projection_below_multi_child( - requirement_map_left, - requirement_map_right, - )?; - - let new_left_size = new_left_child.plan.schema().fields().len(); - // left_schema_mapping does not need to be change, but it is updated with - // those coming form the right side to represent overall join output mapping. - for (idx, field) in - original_right - .schema() - .fields() - .iter() - .enumerate() - .filter(|(idx, field)| { - let right_projection = - collect_columns_in_plan_schema(&new_right_child.plan); - right_projection.contains(&Column::new(field.name(), *idx)) - }) - { - left_schema_mapping.insert( - Column::new(field.name(), idx + left_size), - Column::new(field.name(), idx + new_left_size), - ); - } - for (old, new) in right_schema_mapping.into_iter() { - left_schema_mapping.insert( - Column::new(old.name(), old.index() + left_size), - Column::new(new.name(), new.index() + new_left_size), - ); - } - Ok((new_left_child, new_right_child, left_schema_mapping)) - } - - /// `insert_projection` for windows. - fn insert_projection_below_window( + /// `insert_projection` for windows. + fn insert_projection_below_window( self, w_agg: &WindowAggExec, requirement_map: ColumnRequirements, @@ -3100,8 +2741,8 @@ impl ProjectionOptimizer { /// Responsible for updating the node's plan with new children and possibly updated column indices, /// and for transferring the column mapping to the upper nodes. There is an exception for the - /// projection nodes; they may be removed also in case of being considered as unnecessary, - /// which leads to re-update the mapping after removal. + /// 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 @@ -3165,24 +2806,9 @@ impl ProjectionOptimizer { .collect(), )?; update_mapping(&mut self, all_mappings) - } else if let Some(_cj) = plan_any.downcast_ref::() { - self.plan = self.plan.with_new_children( - self.children_nodes - .iter() - .map(|child| child.plan.clone()) - .collect(), - )?; - update_mapping_cross(&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(filter) = plan_any.downcast_ref::() { + } + // ------------------------------------------------------------------------ + else if let Some(filter) = plan_any.downcast_ref::() { self.plan = rewrite_filter( filter.predicate(), self.children_nodes[0].plan.clone(), @@ -3212,296 +2838,30 @@ impl ProjectionOptimizer { &all_mappings[0], )?; update_mapping(&mut self, all_mappings) - } else if let Some(hj) = plan_any.downcast_ref::() { - 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; - } - } - } else if let Some(nlj) = plan_any.downcast_ref::() { - let left_size = self.children_nodes[0].plan.schema().fields().len(); - let left_mapping = all_mappings.swap_remove(0); - let right_mapping = all_mappings.swap_remove(0); - let new_mapping = left_mapping - .iter() - .map(|(initial, new)| (initial.clone(), new.clone())) - .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, + } else if let Some(projection) = plan_any.downcast_ref::() { + self.plan = rewrite_projection( + projection, self.children_nodes[0].plan.clone(), - self.children_nodes[1].plan.clone(), - &left_mapping, - &right_mapping, - left_size, + &all_mappings[0], )?; - 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) - } else if let Some(smj) = plan_any.downcast_ref::() { - let left_size = self.children_nodes[0].plan.schema().fields().len(); - let left_mapping = all_mappings.swap_remove(0); - let right_mapping = all_mappings.swap_remove(0); - let new_mapping = left_mapping - .iter() - .map(|(initial, new)| (initial.clone(), new.clone())) - .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, + // 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(), )?; - 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) + 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::() { - 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) + 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() @@ -3594,75 +2954,354 @@ impl ProjectionOptimizer { self.plan = res?; } - - Ok(Transformed::yes(self)) + + 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) } - /// 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); + 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) + } - // 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); + 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) + } - // 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 + 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() - .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 + .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() - .map(|(expected, updated)| { - ( - expected, - Column::new( - updated.name(), - projection_columns[updated.index()].index(), - ), - ) - }) - .collect() + .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) } } @@ -3772,12 +3411,6 @@ fn collect_left_used_columns( } /// Collects all fields of a schema from a given execution plan and converts them into a [`IndexSet`] of [`Column`]. -/// -/// # Arguments -/// * `plan`: Reference to an Arc of an ExecutionPlan trait object. -/// -/// # Returns -/// A `IndexSet` containing all columns from the plan's schema. fn collect_columns_in_plan_schema(plan: &Arc) -> IndexSet { plan.schema() .fields() @@ -3789,16 +3422,6 @@ fn collect_columns_in_plan_schema(plan: &Arc) -> IndexSet` containing all columns from the join conditions. fn collect_columns_in_join_conditions( on: &[(PhysicalExprRef, PhysicalExprRef)], filter: Option<&JoinFilter>, @@ -4669,17 +4292,6 @@ fn update_hj_projection_right( } /// Rewrites a filter execution plan with updated column indices. -/// -/// This function updates the column indices in a filter's predicate based on a provided mapping. -/// It creates a new `FilterExec` with the updated predicate. -/// -/// # Arguments -/// * `predicate` - The predicate expression of the filter. -/// * `input_plan` - The input execution plan on which the filter is applied. -/// * `mapping` - An IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `FilterExec` wrapped in an `Arc`. fn rewrite_filter( predicate: &Arc, input_plan: Arc, @@ -4728,18 +4340,6 @@ fn rewrite_hj_filter( } /// Rewrites a projection execution plan with updated column indices. -/// -/// This function updates the column indices in a projection based on a provided mapping. -/// It creates a new `ProjectionExec` with the updated expressions. -/// -/// # Arguments -/// * `projection` - The original projection execution plan. -/// * `input_plan` - The input execution plan on which the projection is applied. -/// * `mapping` - An IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `ProjectionExec` wrapped in an `Arc`. -/// fn rewrite_projection( projection: &ProjectionExec, input_plan: Arc, @@ -4757,17 +4357,6 @@ fn rewrite_projection( } /// Rewrites a repartition execution plan with updated column indices. -/// -/// Updates the partitioning expressions in a repartition plan based on the provided column index mappings. -/// Supports updating the `Partitioning::Hash` variant of partitioning. -/// -/// # Arguments -/// * `partitioning` - The original partitioning strategy. -/// * `input_plan` - The input execution plan on which repartitioning is applied. -/// * `mapping` - An IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `RepartitionExec` wrapped in an `Arc`. fn rewrite_repartition( partitioning: &Partitioning, input_plan: Arc, @@ -4783,17 +4372,6 @@ fn rewrite_repartition( } /// Rewrites a sort execution plan with updated column indices. -/// -/// This function updates the column indices in a sort's expressions based on a provided mapping. -/// It creates a new `SortExec` with the updated expressions. -/// -/// # Arguments -/// * `sort` - The original sort execution plan. -/// * `input_plan` - The input execution plan on which sorting is applied. -/// * `mapping` - An IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `SortExec` wrapped in an `Arc`. fn rewrite_sort( sort: &SortExec, input_plan: Arc, @@ -4808,16 +4386,6 @@ fn rewrite_sort( } /// Rewrites a sort preserving merge execution plan with updated column indices. -/// -/// Updates the sort expressions in a sort preserving merge plan based on the provided column index mappings. -/// -/// # Arguments -/// * `sort` - The original `SortPreservingMergeExec` plan. -/// * `input_plan` - The input execution plan to which the sort preserving merge is applied. -/// * `mapping` - An IndexMap with old and new column index mappings. -/// -/// # Returns -/// A `Result` containing the new `SortPreservingMergeExec` wrapped in an `Arc`. fn rewrite_sort_preserving_merge( sort: &SortPreservingMergeExec, input_plan: Arc, @@ -4878,20 +4446,8 @@ fn rewrite_nested_loop_join( .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. -/// -/// # Arguments -/// * `smj`: The original SortMergeJoinExec to be rewritten. -/// * `left_input_plan`: The updated execution plan for the left input. -/// * `right_input_plan`: The updated execution plan for the right input. -/// * `left_mapping`: Column mapping for the left input. -/// * `right_mapping`: Column mapping for the right input. -/// * `left_size`: The size of the left input, necessary for index calculations. -/// -/// # Returns -/// A Result containing the rewritten execution plan as an Arc, or an error on failure. +/// 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, @@ -4945,20 +4501,8 @@ fn rewrite_sort_merge_join( .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. -/// -/// # Arguments -/// * `shj`: The SymmetricHashJoinExec to be modified. -/// * `left_input_plan`: New execution plan for the left side. -/// * `right_input_plan`: New execution plan for the right side. -/// * `left_mapping`: Mapping for updating left side columns. -/// * `right_mapping`: Mapping for updating right side columns. -/// * `left_size`: Size of the left input for index adjustments. -/// -/// # Returns -/// A Result containing the updated execution plan within an Arc, or an error if the operation fails. +/// 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, @@ -5033,17 +4577,8 @@ fn rewrite_symmetric_hash_join( .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. -/// -/// # Arguments -/// * `agg`: The original AggregateExec to be rewritten. -/// * `input_plan`: The new execution plan to use as input. -/// * `mapping`: A mapping from old to new columns. -/// -/// # Returns -/// A Result that either contains an Option with the new execution plan wrapped in an Arc, or None if no rewriting is possible, along with an error on failure. +/// 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, @@ -5088,17 +4623,8 @@ fn rewrite_aggregate( .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. -/// -/// # Arguments -/// * `w_agg`: The WindowAggExec to be updated. -/// * `input_plan`: The new execution plan to be used. -/// * `mapping`: Column mapping for updating window expressions. -/// -/// # Returns -/// A Result containing either an Option with the updated execution plan in an Arc, or None if rewriting isn't feasible, and an error on failure. +/// 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, @@ -5115,17 +4641,8 @@ fn rewrite_window_aggregate( .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. -/// -/// # Arguments -/// * `bw_agg`: The original BoundedWindowAggExec to be rewritten. -/// * `input_plan`: The new execution plan to use. -/// * `mapping`: Mapping for updating window expressions. -/// -/// # Returns -/// A Result containing an Option with the new execution plan wrapped in an Arc, or None if the rewrite is not possible, and an error on failure. +/// 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, @@ -5147,39 +4664,10 @@ fn rewrite_bounded_window_aggregate( .map(|plan| Some(Arc::new(plan) as _)) } -fn split_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 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_schema_agnostic(plan: &Arc) -> bool { +fn is_plan_schema_agnostic(plan: &Arc) -> bool { let plan_any = plan.as_any(); plan_any.downcast_ref::().is_some() || plan_any.downcast_ref::().is_some() @@ -5237,7 +4725,6 @@ fn analyze_requirements(node: &ProjectionOptimizer) -> ColumnRequirements { /// 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. -/// Each column in the requirement maps corresponds to its own table schema index, not to the join output schema. /// /// # Arguments /// * `left_child`: Reference to the execution plan of the left child. @@ -5249,7 +4736,6 @@ fn analyze_requirements(node: &ProjectionOptimizer) -> ColumnRequirements { /// 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, @@ -5263,6 +4749,7 @@ fn analyze_requirements_of_joins( .into_iter() .map(|col| Column::new(col.name(), col.index() + left_size)), ); + let requirement_map = columns_in_schema .into_iter() .map(|col| { @@ -5277,6 +4764,7 @@ fn analyze_requirements_of_joins( 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)) @@ -5299,6 +4787,51 @@ fn all_input_columns_required( .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: @@ -5421,13 +4954,6 @@ fn window_agg_required( .any(|(_column, used)| *used) } -/// Updates a source provider's projected columns according to the given -/// projection operator's expressions. To use this function safely, one must -/// ensure that all expressions are `Column` expressions without aliases. -fn new_projections_for_columns(projection: &[Column], source: &[usize]) -> Vec { - projection.iter().map(|col| source[col.index()]).collect() -} - /// 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 @@ -5495,13 +5021,6 @@ fn extend_left_mapping_with_right( } /// Calculates the count of removed (unused) columns that precede a given column index. -/// -/// # Arguments -/// * `requirement_map`: Reference to a ColumnRequirements map. -/// * `column_index`: The index of the column in question. -/// -/// # Returns -/// The number of removed columns before the given column index. fn removed_column_count( requirement_map: &ColumnRequirements, column_index: usize, @@ -5563,22 +5082,6 @@ fn index_changes_after_projection_removal( .collect() } -/// 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) -} - #[cfg(test)] mod tests { use super::*; @@ -6319,7 +5822,6 @@ mod tests { #[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(); From c63be56e4d88857ff65bfb9f156ae3be52f4ae3d Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 24 Apr 2024 11:21:14 +0300 Subject: [PATCH 64/85] Review --- datafusion/core/src/physical_optimizer/optimizer.rs | 2 +- datafusion/physical-expr/src/equivalence/projection.rs | 5 +++-- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/projection.rs | 6 +++--- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index a564d21ff2c2..ecb2281f755d 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -120,7 +120,7 @@ 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()), - // OptimizeProjections rule aims achieving the most effective use of projections + // 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()), diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 4227471d7e93..f3a769f5f12a 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{internal_err, Result}; use crate::expressions::Column; use crate::PhysicalExpr; @@ -67,7 +67,8 @@ impl ProjectionMapping { let idx = col.index(); let matching_input_field = input_schema.field(idx); if col.name() != matching_input_field.name() { - return Err(DataFusionError::Internal(format!("Input field name {} does not match with the projection expression {}",matching_input_field.name(),col.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); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 720125628fac..768d30da4d02 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -241,7 +241,7 @@ impl From for SendableRecordBatchStream { } /// Hash aggregate execution plan -#[derive(Debug, Clone)] +#[derive(Debug)] pub struct AggregateExec { /// Aggregation mode (full, partial) mode: AggregateMode, diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 8d204ca5a283..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( @@ -83,9 +83,9 @@ impl ProjectionExec { ); Ok(field) }) - .collect(); + .collect::>>()?; let schema = Arc::new(Schema::new_with_metadata( - fields?, + fields, input_schema.metadata().clone(), )); From 21284755cc108b4037f9435ef4435ac5e4bcc653 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 29 Apr 2024 09:59:27 +0300 Subject: [PATCH 65/85] before test complete --- .../optimize_projections.rs | 157 +++++++++++++----- datafusion/core/src/physical_planner.rs | 9 +- .../src/equivalence/projection.rs | 4 +- datafusion/sqllogictest/test_files/joins.slt | 25 +-- .../sqllogictest/test_files/tpch/q11.slt.part | 48 +++--- .../sqllogictest/test_files/tpch/q22.slt.part | 41 ++--- 6 files changed, 178 insertions(+), 106 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 294fd4446175..e9f8dd47afaf 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3351,14 +3351,16 @@ impl PhysicalOptimizerRule for OptimizeProjections { 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(&plan); + let initial_requirements = + collect_columns_in_plan_schema(&final_schema_determinant); - let mut optimizer = ProjectionOptimizer::new_default(plan); + let mut optimizer = ProjectionOptimizer::new_default(final_schema_determinant); // Insert the initial requirements to the root node, and run the rule. optimizer.required_columns = initial_requirements.clone(); - let mut optimized = optimizer.transform_down(&|o| { + 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 @@ -3366,7 +3368,11 @@ impl PhysicalOptimizerRule for OptimizeProjections { // optimized plan satisfies the initial schema order. optimized = optimized .map_data(|node| satisfy_initial_schema(node, initial_requirements))?; - Ok(optimized.data.plan) + + let new_child = optimized.data.plan; + let new_plan = change_children_final_schema_determinant(plan, new_child)?; + + Ok(new_plan) } fn name(&self) -> &str { @@ -3378,6 +3384,74 @@ impl PhysicalOptimizerRule for OptimizeProjections { } } +fn find_final_schema_determinant( + plan: &Arc, +) -> Arc { + let plan_any = plan.as_any(); + + if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else { + plan.children() + .get(0) + .map(find_final_schema_determinant) + .unwrap_or(plan.clone()) + } +} + +fn change_children_final_schema_determinant( + plan: Arc, + new_child: Arc, +) -> Result> { + let children = plan.children(); + let Some(child) = children.get(0) else { + return Ok(plan.clone()); + }; + let child_any = child.as_any(); + + if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else { + plan.children() + .get(0) + .map(|c| change_children_final_schema_determinant(c.clone(), new_child)) + .transpose() + .map(|new_plan| new_plan.unwrap_or(plan.clone())) + } +} + /// Filters the expressions of a [`ProjectionExec`] according to the given used column indices. fn collect_used_columns( projection_exprs: &[(Arc, String)], @@ -3708,42 +3782,37 @@ fn update_expr_with_projection( sync_with_child: bool, ) -> Result>> { let mut state = RewriteState::Unchanged; - let new_expr = expr - .clone() - .transform_up_mut(&mut |expr: Arc| { - if state == RewriteState::RewrittenInvalid { - return Ok(Transformed::no(expr)); - } - let Some(column) = expr.as_any().downcast_ref::() else { - return Ok(Transformed::no(expr)); - }; - if sync_with_child { - state = RewriteState::RewrittenValid; - // Update the index of `column`: - Ok(Transformed::yes(projected_exprs[column.index()].0.clone())) - } else { - // default to invalid, in case we can't find the relevant column - state = RewriteState::RewrittenInvalid; - // Determine how to update `column` to accommodate `projected_exprs` - projected_exprs - .iter() - .enumerate() - .find_map(|(index, (projected_expr, alias))| { - projected_expr.as_any().downcast_ref::().and_then( - |projected_column| { - column.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)), + 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)) } @@ -3845,7 +3914,7 @@ fn update_column_index( let mut state = RewriteState::Unchanged; let new_expr = expr .clone() - .transform_up_mut(&mut |expr: Arc| { + .transform_up(|expr: Arc| { if state == RewriteState::RewrittenInvalid { return Ok(Transformed::no(expr)); } @@ -3878,7 +3947,7 @@ fn update_equivalence_conditions( ( left_col .clone() - .transform_up_mut(&mut |expr: Arc| { + .transform_up(|expr: Arc| { if left_state == RewriteState::RewrittenInvalid { return Ok(Transformed::no(expr)); } @@ -3899,7 +3968,7 @@ fn update_equivalence_conditions( .data, right_col .clone() - .transform_up_mut(&mut |expr: Arc| { + .transform_up(|expr: Arc| { if right_state == RewriteState::RewrittenInvalid { return Ok(Transformed::no(expr)); } @@ -4078,7 +4147,7 @@ fn update_proj_exprs( .map(|(expr, alias)| { let new_expr = expr .clone() - .transform_up_mut(&mut |expr: Arc| { + .transform_up(|expr: Arc| { let Some(column) = expr.as_any().downcast_ref::() else { return Ok(Transformed::no(expr)); }; @@ -4843,7 +4912,7 @@ fn caching_projections( 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(&mut |expr| { + expr.apply(|expr| { Ok({ if let Some(column) = expr.as_any().downcast_ref::() { *column_ref_map.entry(column.clone()).or_default() += 1; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 9cd701ed7d2d..25cc9d7c7e12 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -34,15 +34,15 @@ use crate::datasource::physical_plan::FileSinkConfig; use crate::datasource::source_as_provider; use crate::error::{DataFusionError, Result}; use crate::execution::context::{ExecutionProps, SessionState}; +use crate::logical_expr::expr_vec_fmt; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ Aggregate, EmptyRelation, Join, Projection, Sort, TableScan, Unnest, Window, }; use crate::logical_expr::{ - Expr, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, - UserDefinedLogicalNode, + Expr, Limit, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, + UserDefinedLogicalNode, Values, }; -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,10 +87,9 @@ 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::{ - expr_vec_fmt, DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, + DescribeTable, DmlStatement, Extension, Filter, RecursiveQuery, ScalarFunctionDefinition, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; use datafusion_physical_expr::expressions::Literal; diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index da4c285d1eba..fe80a86f47f9 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -58,7 +58,7 @@ impl ProjectionMapping { .map(|(expr_idx, (expression, name))| { let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; expression - .transform_down(&|e| match e.as_any().downcast_ref::() { + .transform_down(&|e :Arc | match e.as_any().downcast_ref::() { Some(col) => { // Sometimes, an expression and its name in the input_schema // doesn't match. This can cause problems, so we make sure @@ -72,7 +72,7 @@ impl ProjectionMapping { } let matching_input_column = Column::new(matching_input_field.name(), idx); - Ok(Transformed::yes(Arc::new(matching_input_column))) + Ok(Transformed::yes(Arc::new(matching_input_column) as _)) } None => Ok(Transformed::no(e)), }) diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 80652d7984eb..00a1f2f5c063 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2010,7 +2010,7 @@ set datafusion.explain.logical_plan_only = false; statement ok set datafusion.execution.target_partitions = 4; -# Planning inner nested loop join +# Planning inner nested loop join # inputs are swapped due to inexact statistics + join reordering caused additional projection query TT @@ -2028,17 +2028,18 @@ logical_plan 05)----Filter: join_t2.t2_int > UInt32(1) 06)------TableScan: join_t2 projection=[t2_id, t2_int] physical_plan -01)NestedLoopJoinExec: join_type=Inner, filter=t1_id@0 > t2_id@1 -02)--CoalesceBatchesExec: target_batch_size=2 -03)----FilterExec: t1_id@0 > 10 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -05)--------MemoryExec: partitions=1, partition_sizes=[1] -06)--CoalescePartitionsExec -07)----CoalesceBatchesExec: target_batch_size=2 -08)------ProjectionExec: expr=[t2_id@0 as t2_id] -09)--------FilterExec: t2_int@1 > 1 -10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)------------MemoryExec: partitions=1, partition_sizes=[1] +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)------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] +09)----CoalesceBatchesExec: target_batch_size=2 +10)------FilterExec: t1_id@0 > 10 +11)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------MemoryExec: partitions=1, partition_sizes=[1] query II SELECT join_t1.t1_id, join_t2.t2_id diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index ea678eb255df..a305f4c82a38 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -103,30 +103,30 @@ physical_plan 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 -30)----------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -31)------------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -32)--------------CoalescePartitionsExec -33)----------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -34)------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -36)----------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -38)--------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -40)------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -42)----------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false -43)------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -45)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)--------------------------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 +30)--------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +31)----------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)------------CoalescePartitionsExec +33)--------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +34)----------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +36)--------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +38)------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +40)----------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +42)--------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +45)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)------------------------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/q22.slt.part b/datafusion/sqllogictest/test_files/tpch/q22.slt.part index 1194659252ff..58095ba33652 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -83,25 +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)----------------------------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") }]) -16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------------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 -18)----------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -20)--------------------------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 -21)------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] -22)--------------------CoalescePartitionsExec -23)----------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] -24)------------------------CoalesceBatchesExec: target_batch_size=8192 -25)--------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] -26)----------------------------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") }]) -27)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +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)------------------------------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 From bb06baf8751b06ee07c91dd8ae7b11cfe37af5ad Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 29 Apr 2024 13:26:17 +0300 Subject: [PATCH 66/85] joins are fixed --- .../optimize_projections.rs | 81 ++++++++++++++----- .../sqllogictest/test_files/group_by.slt | 12 +-- datafusion/sqllogictest/test_files/joins.slt | 8 +- .../sqllogictest/test_files/tpch/q2.slt.part | 8 +- 4 files changed, 77 insertions(+), 32 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index e9f8dd47afaf..a1c30838e45c 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -171,6 +171,16 @@ impl ProjectionOptimizer { }; } + // 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()? { @@ -207,16 +217,6 @@ impl ProjectionOptimizer { }; } - // 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, - }; - } - // 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. @@ -3370,9 +3370,12 @@ impl PhysicalOptimizerRule for OptimizeProjections { .map_data(|node| satisfy_initial_schema(node, initial_requirements))?; let new_child = optimized.data.plan; - let new_plan = change_children_final_schema_determinant(plan, new_child)?; - - Ok(new_plan) + if is_plan_schema_determinant(&plan) { + Ok(new_child) + } else { + let x = update_children(plan, new_child)?; + Ok(x) + } } fn name(&self) -> &str { @@ -3384,6 +3387,36 @@ impl PhysicalOptimizerRule for OptimizeProjections { } } +fn is_plan_schema_determinant(plan: &Arc) -> bool { + let plan_any = plan.as_any(); + + if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else if plan_any.downcast_ref::().is_some() { + true + } else { + false + } +} + fn find_final_schema_determinant( plan: &Arc, ) -> Arc { @@ -3407,6 +3440,10 @@ fn find_final_schema_determinant( return plan.clone(); } else if plan_any.downcast_ref::().is_some() { return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); + } else if plan_any.downcast_ref::().is_some() { + return plan.clone(); } else { plan.children() .get(0) @@ -3415,7 +3452,7 @@ fn find_final_schema_determinant( } } -fn change_children_final_schema_determinant( +fn update_children( plan: Arc, new_child: Arc, ) -> Result> { @@ -3443,12 +3480,18 @@ fn change_children_final_schema_determinant( plan.with_new_children(vec![new_child]) } else if child_any.downcast_ref::().is_some() { plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) + } else if child_any.downcast_ref::().is_some() { + plan.with_new_children(vec![new_child]) } else { - plan.children() + let new_child = plan + .children() .get(0) - .map(|c| change_children_final_schema_determinant(c.clone(), new_child)) + .map(|c| update_children(c.clone(), new_child)) .transpose() - .map(|new_plan| new_plan.unwrap_or(plan.clone())) + .map(|new_plan| new_plan.unwrap_or(plan.clone()))?; + plan.with_new_children(vec![new_child]) } } @@ -3547,7 +3590,9 @@ fn collect_column_indices_in_proj_exprs( .iter() .flat_map(|(expr, _)| collect_columns(expr)) .map(|col| col.index()) - .collect::>() + .collect::>() + .into_iter() + .collect() } /// Collects the columns that the projection requires from its input. diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index e333e70483cf..8c46b0b013c0 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -3420,9 +3420,9 @@ logical_plan 08)----------SubqueryAlias: r 09)------------TableScan: sales_global_with_pk projection=[sn, amount] physical_plan -01)ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] -02)--SortPreservingMergeExec: [sn@0 ASC NULLS LAST] -03)----SortExec: expr=[sn@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [sn@0 ASC NULLS LAST] +02)--SortExec: expr=[sn@0 ASC NULLS LAST] +03)----ProjectionExec: expr=[sn@0 as sn, SUM(l.amount)@2 as SUM(l.amount), amount@1 as amount] 04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, amount@1 as amount], aggr=[SUM(l.amount)] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 @@ -3568,9 +3568,9 @@ logical_plan 07)------------SubqueryAlias: l 08)--------------TableScan: sales_global_with_pk projection=[zip_code, country, sn, ts, currency, amount] physical_plan -01)ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] -02)--SortPreservingMergeExec: [sn@0 ASC NULLS LAST] -03)----SortExec: expr=[sn@0 ASC NULLS LAST] +01)SortPreservingMergeExec: [sn@2 ASC NULLS LAST] +02)--SortExec: expr=[sn@2 ASC NULLS LAST] +03)----ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount] 04)------AggregateExec: mode=FinalPartitioned, gby=[sn@0 as sn, zip_code@1 as zip_code, country@2 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 00a1f2f5c063..b095b0d49c58 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1452,8 +1452,8 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@2 as t1_name, t1_int@3 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1478,8 +1478,8 @@ logical_plan 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan 01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@2 as t1_name, t1_int@3 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index f78ff0b6e8ec..c7c8e546f136 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -100,10 +100,10 @@ logical_plan 35)--------------------Filter: region.r_name = Utf8("EUROPE") 36)----------------------TableScan: region projection=[r_regionkey, r_name], partial_filters=[region.r_name = Utf8("EUROPE")] physical_plan -01)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] -02)--GlobalLimitExec: skip=0, fetch=10 -03)----SortPreservingMergeExec: [s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST], fetch=10 -04)------SortExec: TopK(fetch=10), expr=[s_acctbal@5 DESC,n_name@7 ASC NULLS LAST,s_name@2 ASC NULLS LAST,p_partkey@0 ASC NULLS LAST] +01)GlobalLimitExec: skip=0, fetch=10 +02)--SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 +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] +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@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 From ee8dfa65d61e4bd819d7b0b74dc3221af2861279 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 29 Apr 2024 13:59:54 +0300 Subject: [PATCH 67/85] Update optimize_projections.rs --- .../optimize_projections.rs | 91 ++++--------------- 1 file changed, 17 insertions(+), 74 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index a1c30838e45c..9150874d1647 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3390,63 +3390,27 @@ impl PhysicalOptimizerRule for OptimizeProjections { fn is_plan_schema_determinant(plan: &Arc) -> bool { let plan_any = plan.as_any(); - if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else if plan_any.downcast_ref::().is_some() { - true - } else { - false - } + plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() + | plan_any.downcast_ref::().is_some() } fn find_final_schema_determinant( plan: &Arc, ) -> Arc { - let plan_any = plan.as_any(); - - if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); - } else if plan_any.downcast_ref::().is_some() { - return plan.clone(); + if is_plan_schema_determinant(plan) { + plan.clone() } else { plan.children() - .get(0) + .first() .map(find_final_schema_determinant) .unwrap_or(plan.clone()) } @@ -3457,37 +3421,16 @@ fn update_children( new_child: Arc, ) -> Result> { let children = plan.children(); - let Some(child) = children.get(0) else { + let Some(child) = children.first() else { return Ok(plan.clone()); }; - let child_any = child.as_any(); - if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { - plan.with_new_children(vec![new_child]) - } else if child_any.downcast_ref::().is_some() { + if is_plan_schema_determinant(child) { plan.with_new_children(vec![new_child]) } else { let new_child = plan .children() - .get(0) + .first() .map(|c| update_children(c.clone(), new_child)) .transpose() .map(|new_plan| new_plan.unwrap_or(plan.clone()))?; From 16d5e5ee99cba8ada8883151387307c2b1e79f3f Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 29 Apr 2024 14:08:15 +0300 Subject: [PATCH 68/85] Ready for review --- .../src/physical_optimizer/optimize_projections.rs | 14 +++++++------- datafusion/sqllogictest/test_files/select.slt | 1 + .../sqllogictest/test_files/tpch/q3.slt.part | 2 ++ .../sqllogictest/test_files/tpch/q8.slt.part | 2 ++ .../sqllogictest/test_files/tpch/q9.slt.part | 2 ++ datafusion/sqllogictest/test_files/window.slt | 4 ++++ 6 files changed, 18 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9150874d1647..4c187fe31a99 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -3368,13 +3368,12 @@ impl PhysicalOptimizerRule for OptimizeProjections { // 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 { - let x = update_children(plan, new_child)?; - Ok(x) + update_children(plan, new_child) } } @@ -3387,6 +3386,8 @@ impl PhysicalOptimizerRule for OptimizeProjections { } } +/// 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(); @@ -3403,6 +3404,7 @@ fn is_plan_schema_determinant(plan: &Arc) -> bool { | plan_any.downcast_ref::().is_some() } +/// Given a plan, the function returns the closest node to the root which updates the schema. fn find_final_schema_determinant( plan: &Arc, ) -> Arc { @@ -3416,6 +3418,8 @@ fn find_final_schema_determinant( } } +/// 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, @@ -5266,7 +5270,6 @@ mod tests { ], DataType::Int32, None, - false, )), Arc::new(CaseExpr::try_new( Some(Arc::new(Column::new("d", 2))), @@ -5335,7 +5338,6 @@ mod tests { ], DataType::Int32, None, - false, )), Arc::new(CaseExpr::try_new( Some(Arc::new(Column::new("d", 3))), @@ -5407,7 +5409,6 @@ mod tests { ], DataType::Int32, None, - false, )), Arc::new(CaseExpr::try_new( Some(Arc::new(Column::new("d", 2))), @@ -5476,7 +5477,6 @@ mod tests { ], DataType::Int32, None, - false, )), Arc::new(CaseExpr::try_new( Some(Arc::new(Column::new("d_new", 3))), diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index c9c04ebdb97d..a3a4b3bfc584 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1406,6 +1406,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + statement ok drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index bb20789f35c0..e8d352090a24 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -92,6 +92,8 @@ physical_plan 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 + + query IRDI select l_orderkey, diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 506c68c83f5b..c8d67d0d4905 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -163,6 +163,8 @@ physical_plan 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 + + query RR select o_year, diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 687926013ad4..82480464370c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -128,6 +128,8 @@ physical_plan 50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false + + query TRR select nation, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3179a86a93df..c9207a3034b3 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1611,6 +1611,7 @@ physical_plan 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true + query III SELECT c9, @@ -1704,6 +1705,7 @@ physical_plan 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true + query III SELECT c3, SUM(c9) OVER(ORDER BY c3+c4 DESC, c9 DESC, c2 ASC) as sum1, @@ -1804,6 +1806,7 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true + query III SELECT c3, SUM(c9) OVER(ORDER BY c3 DESC, c9 DESC, c2 ASC) as sum1, @@ -2090,6 +2093,7 @@ 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, From c87dbde8fc1a285bb0bbc73e6dc2725da65b8fe1 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 29 Apr 2024 14:38:31 +0300 Subject: [PATCH 69/85] Investigation Projections are preferred to be on top of 1) GlobalLimitExec 2) LocalLimitExec 3) CoalesceBatchesExec Only slt tests are modified. Unit test results have not been updated. --- .../optimize_projections.rs | 8 +- datafusion/sqllogictest/test_files/cte.slt | 4 +- .../sqllogictest/test_files/explain.slt | 4 +- datafusion/sqllogictest/test_files/expr.slt | 2 +- .../sqllogictest/test_files/functions.slt | 1 - .../sqllogictest/test_files/group_by.slt | 25 +-- .../join_disable_repartition_joins.slt | 17 +- datafusion/sqllogictest/test_files/joins.slt | 148 +++++++++-------- datafusion/sqllogictest/test_files/limit.slt | 6 +- .../sqllogictest/test_files/predicates.slt | 63 ++++---- .../sqllogictest/test_files/subquery.slt | 24 +-- .../sqllogictest/test_files/tpch/q1.slt.part | 5 +- .../sqllogictest/test_files/tpch/q10.slt.part | 59 +++---- .../sqllogictest/test_files/tpch/q11.slt.part | 90 ++++++----- .../sqllogictest/test_files/tpch/q12.slt.part | 21 +-- .../sqllogictest/test_files/tpch/q13.slt.part | 23 +-- .../sqllogictest/test_files/tpch/q14.slt.part | 23 +-- .../sqllogictest/test_files/tpch/q15.slt.part | 66 ++++---- .../sqllogictest/test_files/tpch/q16.slt.part | 37 ++--- .../sqllogictest/test_files/tpch/q17.slt.part | 38 ++--- .../sqllogictest/test_files/tpch/q18.slt.part | 48 +++--- .../sqllogictest/test_files/tpch/q19.slt.part | 27 ++-- .../sqllogictest/test_files/tpch/q2.slt.part | 151 +++++++++--------- .../sqllogictest/test_files/tpch/q20.slt.part | 87 +++++----- .../sqllogictest/test_files/tpch/q21.slt.part | 96 +++++------ .../sqllogictest/test_files/tpch/q22.slt.part | 39 ++--- .../sqllogictest/test_files/tpch/q3.slt.part | 44 ++--- .../sqllogictest/test_files/tpch/q4.slt.part | 25 +-- .../sqllogictest/test_files/tpch/q5.slt.part | 89 ++++++----- .../sqllogictest/test_files/tpch/q6.slt.part | 4 +- .../sqllogictest/test_files/tpch/q7.slt.part | 90 ++++++----- .../sqllogictest/test_files/tpch/q8.slt.part | 124 +++++++------- .../sqllogictest/test_files/tpch/q9.slt.part | 80 +++++----- datafusion/sqllogictest/test_files/union.slt | 43 ++--- datafusion/sqllogictest/test_files/window.slt | 111 ++++++------- 35 files changed, 894 insertions(+), 828 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 4c187fe31a99..ba445ea504cb 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -4730,10 +4730,10 @@ fn rewrite_bounded_window_aggregate( /// the order with projections without any further adaptation. fn is_plan_schema_agnostic(plan: &Arc) -> bool { let plan_any = plan.as_any(); - plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() + // plan_any.downcast_ref::().is_some() + // || plan_any.downcast_ref::().is_some() + // || plan_any.downcast_ref::().is_some()|| + plan_any.downcast_ref::().is_some() } /// Checks if the given expression is trivial. diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 1bb10e938fd6..eb84f74b407a 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -739,8 +739,8 @@ physical_plan 04)--ProjectionExec: expr=[2 as val] 05)----CrossJoinExec 06)------CoalescePartitionsExec -07)--------CoalesceBatchesExec: target_batch_size=8182 -08)----------ProjectionExec: expr=[] +07)--------ProjectionExec: expr=[] +08)----------CoalesceBatchesExec: target_batch_size=8182 09)------------FilterExec: val@0 < 2 10)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)----------------WorkTableExec: name=recursive_cte diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 23dc10c51581..a713c4bb916a 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)CoalesceBatchesExec: target_batch_size=8192 -02)--ProjectionExec: expr=[c1@0 as c1] +01)ProjectionExec: expr=[c1@0 as c1] +02)--CoalesceBatchesExec: target_batch_size=8192 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 diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index ff63416b3a10..7e7ebd8529da 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -1897,7 +1897,7 @@ SELECT substring('alphabet' for 1); ---- a -# The 'from' and 'for' parameters don't support string types, because they should be treated as +# The 'from' and 'for' parameters don't support string types, because they should be treated as # regular expressions, which we have not implemented yet. query error DataFusion error: Error during planning: No function matches the given name and argument types SELECT substring('alphabet' FROM '3') diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index c8675a5d9c54..bc8f6a268703 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -1158,4 +1158,3 @@ drop table uuid_table statement ok drop table t - diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 8c46b0b013c0..a84f49affc49 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2875,9 +2875,9 @@ physical_plan 01)SortExec: expr=[sn@2 ASC NULLS LAST] 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)------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] +04)------ProjectionExec: expr=[zip_code@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, amount@3 as amount] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3871,12 +3871,13 @@ logical_plan physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +03)----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +07)----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +08)------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # reset partition number to 8. statement ok @@ -4026,9 +4027,9 @@ 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)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] +01)ProjectionExec: expr=[c@0 as c, c@3 as c, sum1@2 as sum1, sum1@5 as sum1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)] 04)------ProjectionExec: expr=[c@0 as c, b@1 as b, SUM(multiple_ordered_table_with_pk.d)@2 as sum1] 05)--------AggregateExec: mode=Single, gby=[c@1 as c, b@0 as b], aggr=[SUM(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index f33a04ce98c9..3d7d3cfb0e53 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,11 +57,12 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true -06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -07)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +03)----ProjectionExec: expr=[a@1 as a] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true +07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true # preserve_inner_join query IIII nosort @@ -99,9 +100,9 @@ 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)----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] +03)----ProjectionExec: expr=[a@0 as a2, b@1 as b] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] 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 08)------------FilterExec: d@3 = 3 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index b095b0d49c58..8e0bae898a46 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1325,16 +1325,17 @@ logical_plan 05)------TableScan: join_t2 projection=[t2_id] physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------CoalesceBatchesExec: target_batch_size=2 -09)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------MemoryExec: partitions=1, partition_sizes=[1] +02)--ProjectionExec: expr=[t1_id@0 as t1_id] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1352,16 +1353,17 @@ logical_plan physical_plan 01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] 02)--AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] -09)--------CoalesceBatchesExec: target_batch_size=2 -10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +03)----ProjectionExec: expr=[t1_id@0 as t1_id] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] +06)----------CoalesceBatchesExec: target_batch_size=2 +07)------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------MemoryExec: partitions=1, partition_sizes=[1] +10)----------CoalesceBatchesExec: target_batch_size=2 +11)------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +12)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)----------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1384,16 +1386,17 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -07)------------CoalesceBatchesExec: target_batch_size=2 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] -09)----------------CoalesceBatchesExec: target_batch_size=2 -10)------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)----------------------MemoryExec: partitions=1, partition_sizes=[1] -13)----------------CoalesceBatchesExec: target_batch_size=2 -14)------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -15)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -16)----------------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------ProjectionExec: expr=[t1_id@0 as t1_id] +08)--------------CoalesceBatchesExec: target_batch_size=2 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] +10)------------------CoalesceBatchesExec: target_batch_size=2 +11)--------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------------------MemoryExec: partitions=1, partition_sizes=[1] +14)------------------CoalesceBatchesExec: target_batch_size=2 +15)--------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +17)------------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1451,9 +1454,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1477,9 +1480,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] @@ -1508,9 +1511,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1534,9 +1537,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] @@ -1568,7 +1571,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1592,7 +1595,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1621,9 +1624,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1647,9 +1650,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] +01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] @@ -1677,12 +1680,13 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -03)----MemoryExec: partitions=1, partition_sizes=[1] -04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)--------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] +04)------MemoryExec: partitions=1, partition_sizes=[1] +05)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1699,17 +1703,18 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] -07)----CoalesceBatchesExec: target_batch_size=2 -08)------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 -09)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -11)------------MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 +10)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2031,8 +2036,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)------CoalesceBatchesExec: target_batch_size=2 -05)--------ProjectionExec: expr=[t2_id@0 as t2_id] +04)------ProjectionExec: expr=[t2_id@0 as t2_id] +05)--------CoalesceBatchesExec: target_batch_size=2 06)----------FilterExec: t2_int@1 > 1 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3411,12 +3416,13 @@ logical_plan physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] -05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +03)----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +07)----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +08)------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # run query above in multiple partitions statement ok diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 183c2bdc5251..b4138f38ea2b 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)--------GlobalLimitExec: skip=6, fetch=3 -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------ProjectionExec: expr=[] +05)--------ProjectionExec: expr=[] +06)----------GlobalLimitExec: skip=6, fetch=3 +07)------------CoalesceBatchesExec: target_batch_size=8192 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 ebbebc9ea018..ec54e9cdc9f7 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -635,20 +635,21 @@ logical_plan 05)----Filter: (part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) 06)------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] physical_plan -01)CoalesceBatchesExec: target_batch_size=8192 -02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------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 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true -09)----CoalesceBatchesExec: target_batch_size=8192 -10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -11)--------CoalesceBatchesExec: target_batch_size=8192 -12)----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 -13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true +01)ProjectionExec: expr=[l_partkey@0 as l_partkey] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15 +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------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 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true +10)------CoalesceBatchesExec: target_batch_size=8192 +11)--------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +12)----------CoalesceBatchesExec: target_batch_size=8192 +13)------------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -727,24 +728,26 @@ logical_plan 10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] +02)--ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)] +05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true 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 -17)------CoalesceBatchesExec: target_batch_size=8192 -18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -19)----------MemoryExec: partitions=1, partition_sizes=[1] +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 +16)----------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 +17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true +19)--------CoalesceBatchesExec: target_batch_size=8192 +20)----------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +21)------------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index f4c4b5b3599b..2b298277dd9a 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -187,9 +187,9 @@ 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)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] +01)ProjectionExec: expr=[t1_id@2 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@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t2_id@0, 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 @@ -220,9 +220,9 @@ 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)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] +01)ProjectionExec: expr=[t1_id@2 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] 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))] 06)----------CoalesceBatchesExec: target_batch_size=2 @@ -254,9 +254,9 @@ 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)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] +01)ProjectionExec: expr=[t1_id@0 as t1_id, SUM(t2.t2_int)@1 as t2_sum] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 06)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -291,9 +291,9 @@ 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)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] +01)ProjectionExec: expr=[t1_id@2 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@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t2_id@0, 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)] diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 8464fee16e1e..175040420160 100644 --- a/datafusion/sqllogictest/test_files/tpch/q1.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q1.slt.part @@ -56,9 +56,8 @@ 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_discountlineitem.l_discountDecimal128(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)------------------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 +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 query TTRRRRRRRI select diff --git a/datafusion/sqllogictest/test_files/tpch/q10.slt.part b/datafusion/sqllogictest/test_files/tpch/q10.slt.part index 1515040eccd0..d66f5b0f655d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -78,36 +78,39 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] +09)----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +14)--------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)] 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] -19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -21)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)----------------------------------------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)--------------------------------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 -36)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -37)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -38)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +18)----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 +19)------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +24)----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)------------------------------------------------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 +26)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +28)----------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +29)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 +31)----------------------------------------------------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 +32)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +34)------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +35)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------FilterExec: l_returnflag@3 = R +37)------------------------------------------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 +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], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index a305f4c82a38..1c2d344daeae 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -83,50 +83,54 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] +10)------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +15)----------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -18)----------------------------------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, ps_supplycost], has_header=false -19)------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -21)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)--------------------------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 -30)--------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -31)----------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -32)------------CoalescePartitionsExec -33)--------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -34)----------------CoalesceBatchesExec: target_batch_size=8192 -35)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] -36)--------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -38)------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] -40)----------------------------CoalesceBatchesExec: target_batch_size=8192 -41)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -42)--------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false -43)----------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -45)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)------------------------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 +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +20)--------------------------------------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, ps_supplycost], has_header=false +21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +23)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +24)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +25)------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +27)----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +28)------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------------FilterExec: n_name@1 = GERMANY +30)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +32)--------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +33)----------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +34)------------CoalescePartitionsExec +35)--------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +36)----------------ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] +37)------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +39)----------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +41)--------------------------ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] +42)----------------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)] +44)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)----------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +46)------------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false +47)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +48)----------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +49)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +50)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +51)----------------------CoalesceBatchesExec: target_batch_size=8192 +52)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +53)--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +54)----------------------------CoalesceBatchesExec: target_batch_size=8192 +55)------------------------------FilterExec: n_name@1 = GERMANY +56)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +57)----------------------------------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 a32cac928fe2..fd0ebc0cbbd4 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -66,17 +66,18 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------ProjectionExec: expr=[l_shipmode@1 as l_shipmode, o_orderpriority@3 as o_orderpriority] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 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 -17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -18)----------------------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_orderpriority], has_header=false +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------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 +16)------------------------------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 +17)--------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +19)------------------------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_orderpriority], has_header=false query TII diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index e5ea885d9162..0760903470be 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -64,18 +64,19 @@ physical_plan 08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] 09)----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] 10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)----------------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 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 +21)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% +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_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 d53fe0b3177e..b6ca92129cd6 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -45,18 +45,19 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -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)----------------CoalesceBatchesExec: target_batch_size=8192 +05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_type@4 as p_type] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 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 -14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -16)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 +13)------------------------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 +14)--------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +17)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 51d45ba5ff4c..94693ca31e2c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -73,39 +73,41 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] 02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] +03)----ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 +08)--------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false -13)----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -14)------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -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)--------------------------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 -23)----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 -24)------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] -25)--------------CoalescePartitionsExec -26)----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] -27)------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -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)----------------------------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 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +15)--------------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +16)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +17)------------------------CoalesceBatchesExec: target_batch_size=8192 +18)--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +19)----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +20)------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +21)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +23)------------------------------------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 +24)----------CoalesceBatchesExec: target_batch_size=8192 +25)------------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +26)--------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +27)----------------CoalescePartitionsExec +28)------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +29)--------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +30)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +31)------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 +33)----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +34)------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +35)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 +37)------------------------------------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 query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 8c679421ce50..589b161c4569 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -82,24 +82,25 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] -19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -21)----------------------------------------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], has_header=false -22)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -24)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) -26)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)--------------------------------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 +17)--------------------------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +22)------------------------------------------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], has_header=false +23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +25)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) +27)----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +28)------------------------------------------------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 +29)----------------------------CoalesceBatchesExec: target_batch_size=8192 +30)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +31)--------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] +32)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% +34)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +35)----------------------------------------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 4ca2eee20910..3dca369b20ac 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -55,26 +55,28 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] +05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 +08)--------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -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 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +13)------------------------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 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 -19)------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] -20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -21)----------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -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_partkey, l_quantity], has_header=false +15)----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +17)--------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +19)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)--------------------------------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 +21)--------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] +22)----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +23)------------------CoalesceBatchesExec: target_batch_size=8192 +24)--------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +25)----------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +26)------------------------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], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index 7917a1b065e3..2b771156f83c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -75,30 +75,32 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] +09)----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +14)--------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false -19)----------------------------CoalesceBatchesExec: target_batch_size=8192 -20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -21)--------------------------------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_totalprice, o_orderdate], has_header=false -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)----------------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 -30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -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_quantity], has_header=false +16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +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_name], has_header=false +21)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 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_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false +24)----------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +26)--------------------------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 +27)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] +28)------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 +30)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] +31)------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +33)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] +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_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 51150d6f2f50..383d532d9406 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -68,20 +68,21 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND 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)----------------CoalesceBatchesExec: target_batch_size=8192 +05)--------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------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 +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 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 -14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -15)----------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 -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_size, p_container], has_header=false +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------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 +13)------------------------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 +14)--------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +16)------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 +18)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +19)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index c7c8e546f136..ee6341d5960c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -103,84 +103,91 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=10 02)--SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 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] -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] +04)------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@8 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@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] +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, ps_supplycost@7, n_name@8, MIN(partsupp.ps_supplycost)@10, ps_partkey@9] 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 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] -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@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] +09)----------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, ps_supplycost@7 as ps_supplycost, n_name@8 as n_name] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 +14)--------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@5 as s_phone, s_acctbal@6 as s_acctbal, s_comment@7 as s_comment, ps_supplycost@8 as ps_supplycost, n_name@10 as n_name, n_regionkey@11 as n_regionkey] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 +16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)] 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 +18)----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 +19)------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment, ps_supplycost@3 as ps_supplycost] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 +24)----------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] 25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -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 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +29)--------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +30)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +31)------------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS +32)--------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +33)----------------------------------------------------------------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 +34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +36)--------------------------------------------------------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 +37)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)--------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@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/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false +41)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +43)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +44)--------------------------------------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 +45)----------------------CoalesceBatchesExec: target_batch_size=8192 +46)------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +47)--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +48)----------------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------------FilterExec: r_name@1 = EUROPE +50)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +52)------------CoalesceBatchesExec: target_batch_size=8192 +53)--------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 +54)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +55)------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +57)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] +58)------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] +59)--------------------------CoalesceBatchesExec: target_batch_size=8192 +60)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)] +61)------------------------------CoalesceBatchesExec: target_batch_size=8192 +62)--------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 +63)----------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] +64)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +65)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +66)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +67)------------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +68)--------------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] +69)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +70)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] +71)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +72)----------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +73)------------------------------------------------------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 +74)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +75)----------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +76)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +77)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +78)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +79)------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +80)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +81)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +82)------------------------------CoalesceBatchesExec: target_batch_size=8192 +83)--------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +84)----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +85)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +86)--------------------------------------FilterExec: r_name@1 = EUROPE +87)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +88)------------------------------------------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 8e081ac4e94f..af3f538f7a03 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -83,50 +83,53 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[s_name@0 ASC NULLS LAST] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -07)------------CoalesceBatchesExec: target_batch_size=8192 -08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] +03)----ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)] +06)----------CoalesceBatchesExec: target_batch_size=8192 +07)------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +08)--------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------------------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 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 -20)--------CoalesceBatchesExec: target_batch_size=8192 -21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -22)------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] -24)----------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -26)--------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -28)------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -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)----------------------------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 -38)----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -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)--------------------------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 +16)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +17)------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +18)--------------------------CoalesceBatchesExec: target_batch_size=8192 +19)----------------------------FilterExec: n_name@1 = CANADA +20)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +21)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +22)----------CoalesceBatchesExec: target_batch_size=8192 +23)------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +24)--------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] +25)----------------CoalesceBatchesExec: target_batch_size=8192 +26)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1 +27)--------------------CoalesceBatchesExec: target_batch_size=8192 +28)----------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +29)------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +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_availqty], has_header=false +34)----------------------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +36)--------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +37)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)------------------------------------FilterExec: p_name@1 LIKE forest% +39)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +40)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +41)--------------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +42)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +43)------------------------CoalesceBatchesExec: target_batch_size=8192 +44)--------------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 +45)----------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +46)------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] +47)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +48)----------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 +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_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 3d6428882374..959d96f8a693 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -97,54 +97,58 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] +08)--------------ProjectionExec: expr=[s_name@0 as s_name] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +15)----------------------------ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)] 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)--------------------------------------------------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)------------------------------------------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)----------------------------------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 -47)----------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -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)----------------------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 +19)------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 +20)--------------------------------------ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 +25)------------------------------------------------ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] +28)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +30)----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)------------------------------------------------------------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 +32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +34)----------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +35)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +37)----------------------------------------------------------------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 +38)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +40)------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] +41)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------------------------FilterExec: o_orderstatus@1 = F +43)------------------------------------------------------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 +44)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +46)--------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] +47)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------------------------FilterExec: n_name@1 = SAUDI ARABIA +49)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +50)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +51)------------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +53)----------------------------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 +54)--------------------CoalesceBatchesExec: target_batch_size=8192 +55)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +56)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] +57)--------------------------CoalesceBatchesExec: target_batch_size=8192 +58)----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 +59)------------------------------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 58095ba33652..5faaf8a73d6a 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -86,25 +86,26 @@ physical_plan 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)------------------------------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 +13)------------------------ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------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") }]) +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_phone, c_acctbal], has_header=false +22)------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 +24)----------------------------------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 +25)----------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] +26)------------------------CoalescePartitionsExec +27)--------------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] +28)----------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] +29)------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)--------------------------------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") }]) +31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +32)------------------------------------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 e8d352090a24..b2689c6a319f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -67,30 +67,32 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] +09)----------------ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +14)--------------------------ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] 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 -22)----------------------------CoalesceBatchesExec: target_batch_size=8192 -23)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +18)----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +19)------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------FilterExec: c_mktsegment@1 = BUILDING +22)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +23)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false 24)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------------------FilterExec: o_orderdate@2 < 9204 -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)------------------------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 +25)----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +26)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)--------------------------------------FilterExec: o_orderdate@2 < 9204 +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, o_shippriority], 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 +33)------------------------------FilterExec: l_shipdate@3 > 9204 +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_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 929b5cbefa8c..874d5abc201b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -60,20 +60,21 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -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)----------------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 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)----------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 +16)------------------------------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 +17)--------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 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 +20)--------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 +22)------------------------------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 7744a2b6038d..4b4b23e821eb 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -74,52 +74,57 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] +08)--------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +13)------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +18)----------------------------------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)] +21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +23)--------------------------------------------ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)] 26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)------------------------------------------------------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 -37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -38)----------------------------------------------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_extendedprice, l_discount], has_header=false -39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -43)--------------------------CoalesceBatchesExec: target_batch_size=8192 -44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -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, n_regionkey], has_header=false -47)------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -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 +27)----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 +28)------------------------------------------------------ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] +29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] +31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +33)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +34)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +35)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +37)----------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] +38)------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 +40)----------------------------------------------------------------------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 +41)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +43)------------------------------------------------------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_extendedprice, l_discount], has_header=false +44)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +46)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +48)------------------------------CoalesceBatchesExec: target_batch_size=8192 +49)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +50)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)------------------------------------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 +52)--------------------CoalesceBatchesExec: target_batch_size=8192 +53)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +54)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +55)--------------------------CoalesceBatchesExec: target_batch_size=8192 +56)----------------------------FilterExec: r_name@1 = ASIA +57)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +58)--------------------------------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 f3992145a8c1..e54b3c1ccd03 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)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] +06)----------CoalesceBatchesExec: target_batch_size=8192 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/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 99d5c934c9ad..25f42304be4b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -91,53 +91,57 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] +08)--------------ProjectionExec: expr=[n_name@4 as supp_nation, n_name@6 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] +13)------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@6 as n_name] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 +18)----------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 +20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)] 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)----------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 -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_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false -36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -38)------------------------------------------------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], has_header=false -39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -41)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -42)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -43)----------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -45)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -46)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -47)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -49)--------------------CoalesceBatchesExec: target_batch_size=8192 -50)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -51)------------------------CoalesceBatchesExec: target_batch_size=8192 -52)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -53)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -54)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +22)------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 +23)--------------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)] +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +28)------------------------------------------------------ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] +29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] +31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +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_nationkey], has_header=false +35)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +37)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +38)------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 +39)--------------------------------------------------------------------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_extendedprice, l_discount, l_shipdate], has_header=false +40)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +41)----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +42)------------------------------------------------------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], has_header=false +43)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +45)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +46)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +47)------------------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +49)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +51)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +52)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +53)--------------------CoalesceBatchesExec: target_batch_size=8192 +54)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +55)------------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +57)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +58)------------------------------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/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index c8d67d0d4905..69e778646784 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -97,71 +97,77 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] +08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@4 as nation] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] +13)------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +18)----------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 +20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)] 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +22)------------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 +23)--------------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)] +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +28)------------------------------------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] 31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -33)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] -35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -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 -42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -44)------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 -53)------------------------------------------------------------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 -54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -62)----------------------------CoalesceBatchesExec: target_batch_size=8192 -63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -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)------------------------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 +32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +33)----------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] +34)------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +35)--------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)] +36)----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +38)--------------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] +39)----------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] +41)--------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +42)----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +43)------------------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +44)--------------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +45)----------------------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL +46)------------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +47)--------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +48)--------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +49)----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +50)------------------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +51)----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +53)--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +54)----------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +55)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +56)--------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +57)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +58)------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 +59)--------------------------------------------------------------------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 +60)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +61)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +62)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +63)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +64)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +65)------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +66)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +67)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +68)------------------------------CoalesceBatchesExec: target_batch_size=8192 +69)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +70)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +71)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +72)--------------------CoalesceBatchesExec: target_batch_size=8192 +73)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +74)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] +75)--------------------------CoalesceBatchesExec: target_batch_size=8192 +76)----------------------------FilterExec: r_name@1 = AMERICA +77)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +78)--------------------------------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 82480464370c..6a59294526df 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -84,49 +84,53 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] +09)----------------ProjectionExec: expr=[n_name@7 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@4 * l_quantity@0 as amount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] +14)--------------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +19)------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] 20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)] 22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -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 +23)--------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 +24)----------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +29)--------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] 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 -35)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -37)----------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -38)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@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/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -42)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -44)------------------------------------------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 -45)------------------------------CoalesceBatchesExec: target_batch_size=8192 -46)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -47)----------------------------------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], has_header=false -48)----------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +31)------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] +32)--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +33)----------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +34)------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------------------------------FilterExec: p_name@1 LIKE %green% +37)------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)--------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false +39)--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +41)------------------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +42)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +44)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +45)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +46)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +47)--------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +48)----------------------------------------------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 +49)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +51)------------------------------------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], has_header=false +52)----------------------CoalesceBatchesExec: target_batch_size=8192 +53)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +54)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +55)----------------------------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/union.slt b/datafusion/sqllogictest/test_files/union.slt index 6b0d53fb124f..8ae7c5b34135 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -285,27 +285,28 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 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)@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)--CoalesceBatchesExec: target_batch_size=2 -16)----ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] -18)--------CoalesceBatchesExec: target_batch_size=2 -19)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -20)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -21)--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -22)----------------CoalesceBatchesExec: target_batch_size=2 -23)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -24)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -25)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -26)------------------------MemoryExec: partitions=1, partition_sizes=[1] -27)--------CoalesceBatchesExec: target_batch_size=2 -28)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -29)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -30)--------------MemoryExec: partitions=1, partition_sizes=[1] +10)------ProjectionExec: expr=[name@1 as name, CAST(t2.id AS Int32)@2 as CAST(t2.id AS Int32)] +11)--------CoalesceBatchesExec: target_batch_size=2 +12)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +13)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------MemoryExec: partitions=1, partition_sizes=[1] +16)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +17)----CoalesceBatchesExec: target_batch_size=2 +18)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] +19)--------CoalesceBatchesExec: target_batch_size=2 +20)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +21)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +22)--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +23)----------------CoalesceBatchesExec: target_batch_size=2 +24)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +25)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +26)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +27)------------------------MemoryExec: partitions=1, partition_sizes=[1] +28)--------CoalesceBatchesExec: target_batch_size=2 +29)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +30)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +31)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c9207a3034b3..a173db3ea96a 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)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] +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 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] 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,24 +1517,23 @@ 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@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] +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] 02)--GlobalLimitExec: skip=0, fetch=5 -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] -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] -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] -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] -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] -18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true +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] +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] +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] +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] +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] +17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/null_cases.csv]]}, projection=[c1, c2, c3], has_header=true query IIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIIII SELECT @@ -1604,8 +1603,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)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] +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 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] @@ -1648,8 +1647,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)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] +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 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] @@ -1694,8 +1693,8 @@ logical_plan 06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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)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] +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 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.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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] @@ -1753,8 +1752,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)--------------------CoalesceBatchesExec: target_batch_size=4096 -12)----------------------ProjectionExec: expr=[c1@0 as c1] +11)--------------------ProjectionExec: expr=[c1@0 as c1] +12)----------------------CoalesceBatchesExec: target_batch_size=4096 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 @@ -2138,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)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] +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 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 }] @@ -2184,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)GlobalLimitExec: skip=0, fetch=5 -02)--ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] +01)ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2] +02)--GlobalLimitExec: skip=0, fetch=5 03)----SortExec: TopK(fetch=5), expr=[c9@2 ASC NULLS LAST] 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] @@ -2552,8 +2551,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)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] +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 03)----SortExec: TopK(fetch=5), expr=[inc_col@24 DESC] 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] @@ -2714,8 +2713,8 @@ logical_plan 07)------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_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 08)--------------TableScan: annotated_data_finite projection=[ts, inc_col] physical_plan -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] +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 03)----SortExec: TopK(fetch=5), expr=[inc_col@10 ASC NULLS LAST] 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] @@ -2768,8 +2767,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)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] +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 03)----SortExec: TopK(fetch=5), expr=[inc_col@5 ASC NULLS LAST] 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] @@ -2815,12 +2814,13 @@ 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)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] +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] query IIII SELECT @@ -2861,12 +2861,13 @@ 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)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] +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] query IIII SELECT @@ -2960,8 +2961,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)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] +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 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] @@ -3362,8 +3363,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)GlobalLimitExec: skip=0, fetch=5 -02)--ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] +01)ProjectionExec: expr=[min1@0 as min1, max1@1 as max1] +02)--GlobalLimitExec: skip=0, fetch=5 03)----SortExec: TopK(fetch=5), expr=[c3@2 ASC NULLS LAST] 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] @@ -4065,8 +4066,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)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] +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 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] From bd8f2b289dd579ff609a8d926aca70d43b4f2e98 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Mon, 29 Apr 2024 14:39:47 +0300 Subject: [PATCH 70/85] Revert "Investigation" This reverts commit c87dbde8fc1a285bb0bbc73e6dc2725da65b8fe1. --- .../optimize_projections.rs | 8 +- datafusion/sqllogictest/test_files/cte.slt | 4 +- .../sqllogictest/test_files/explain.slt | 4 +- datafusion/sqllogictest/test_files/expr.slt | 2 +- .../sqllogictest/test_files/functions.slt | 1 + .../sqllogictest/test_files/group_by.slt | 25 ++- .../join_disable_repartition_joins.slt | 17 +- datafusion/sqllogictest/test_files/joins.slt | 148 ++++++++--------- datafusion/sqllogictest/test_files/limit.slt | 6 +- .../sqllogictest/test_files/predicates.slt | 63 ++++---- .../sqllogictest/test_files/subquery.slt | 24 +-- .../sqllogictest/test_files/tpch/q1.slt.part | 5 +- .../sqllogictest/test_files/tpch/q10.slt.part | 59 ++++--- .../sqllogictest/test_files/tpch/q11.slt.part | 90 +++++------ .../sqllogictest/test_files/tpch/q12.slt.part | 21 ++- .../sqllogictest/test_files/tpch/q13.slt.part | 23 ++- .../sqllogictest/test_files/tpch/q14.slt.part | 23 ++- .../sqllogictest/test_files/tpch/q15.slt.part | 66 ++++---- .../sqllogictest/test_files/tpch/q16.slt.part | 37 +++-- .../sqllogictest/test_files/tpch/q17.slt.part | 38 +++-- .../sqllogictest/test_files/tpch/q18.slt.part | 48 +++--- .../sqllogictest/test_files/tpch/q19.slt.part | 27 ++-- .../sqllogictest/test_files/tpch/q2.slt.part | 151 +++++++++--------- .../sqllogictest/test_files/tpch/q20.slt.part | 87 +++++----- .../sqllogictest/test_files/tpch/q21.slt.part | 96 ++++++----- .../sqllogictest/test_files/tpch/q22.slt.part | 39 +++-- .../sqllogictest/test_files/tpch/q3.slt.part | 44 +++-- .../sqllogictest/test_files/tpch/q4.slt.part | 25 ++- .../sqllogictest/test_files/tpch/q5.slt.part | 89 +++++------ .../sqllogictest/test_files/tpch/q6.slt.part | 4 +- .../sqllogictest/test_files/tpch/q7.slt.part | 90 +++++------ .../sqllogictest/test_files/tpch/q8.slt.part | 124 +++++++------- .../sqllogictest/test_files/tpch/q9.slt.part | 80 +++++----- datafusion/sqllogictest/test_files/union.slt | 43 +++-- datafusion/sqllogictest/test_files/window.slt | 111 +++++++------ 35 files changed, 828 insertions(+), 894 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index ba445ea504cb..4c187fe31a99 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -4730,10 +4730,10 @@ fn rewrite_bounded_window_aggregate( /// the order with projections without any further adaptation. fn is_plan_schema_agnostic(plan: &Arc) -> bool { let plan_any = plan.as_any(); - // plan_any.downcast_ref::().is_some() - // || plan_any.downcast_ref::().is_some() - // || plan_any.downcast_ref::().is_some()|| - plan_any.downcast_ref::().is_some() + plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() } /// Checks if the given expression is trivial. diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index eb84f74b407a..1bb10e938fd6 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -739,8 +739,8 @@ physical_plan 04)--ProjectionExec: expr=[2 as val] 05)----CrossJoinExec 06)------CoalescePartitionsExec -07)--------ProjectionExec: expr=[] -08)----------CoalesceBatchesExec: target_batch_size=8182 +07)--------CoalesceBatchesExec: target_batch_size=8182 +08)----------ProjectionExec: expr=[] 09)------------FilterExec: val@0 < 2 10)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 11)----------------WorkTableExec: name=recursive_cte diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index a713c4bb916a..23dc10c51581 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 diff --git a/datafusion/sqllogictest/test_files/expr.slt b/datafusion/sqllogictest/test_files/expr.slt index 7e7ebd8529da..ff63416b3a10 100644 --- a/datafusion/sqllogictest/test_files/expr.slt +++ b/datafusion/sqllogictest/test_files/expr.slt @@ -1897,7 +1897,7 @@ SELECT substring('alphabet' for 1); ---- a -# The 'from' and 'for' parameters don't support string types, because they should be treated as +# The 'from' and 'for' parameters don't support string types, because they should be treated as # regular expressions, which we have not implemented yet. query error DataFusion error: Error during planning: No function matches the given name and argument types SELECT substring('alphabet' FROM '3') diff --git a/datafusion/sqllogictest/test_files/functions.slt b/datafusion/sqllogictest/test_files/functions.slt index bc8f6a268703..c8675a5d9c54 100644 --- a/datafusion/sqllogictest/test_files/functions.slt +++ b/datafusion/sqllogictest/test_files/functions.slt @@ -1158,3 +1158,4 @@ drop table uuid_table statement ok drop table t + diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index a84f49affc49..8c46b0b013c0 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2875,9 +2875,9 @@ physical_plan 01)SortExec: expr=[sn@2 ASC NULLS LAST] 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@4 as zip_code, country@5 as country, sn@6 as sn, ts@7 as ts, currency@8 as currency, sn@0 as sn, amount@3 as amount] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1 +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] @@ -3871,13 +3871,12 @@ logical_plan physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -03)----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -07)----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -08)------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # reset partition number to 8. statement ok @@ -4027,9 +4026,9 @@ 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@3 as c, sum1@2 as sum1, sum1@5 as sum1] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(b@1, b@1)] +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]) 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index 3d7d3cfb0e53..f33a04ce98c9 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -57,12 +57,11 @@ logical_plan physical_plan 01)GlobalLimitExec: skip=0, fetch=5 02)--SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5 -03)----ProjectionExec: expr=[a@1 as a] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true -07)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true +06)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true # preserve_inner_join query IIII nosort @@ -100,9 +99,9 @@ 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 -05)--------HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(d@1, d@3), (c@0, c@2)] +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 08)------------FilterExec: d@3 = 3 diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 8e0bae898a46..b095b0d49c58 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1325,17 +1325,16 @@ logical_plan 05)------TableScan: join_t2 projection=[t2_id] physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[] -02)--ProjectionExec: expr=[t1_id@0 as t1_id] -03)----CoalesceBatchesExec: target_batch_size=2 -04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -05)--------CoalesceBatchesExec: target_batch_size=2 -06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -08)--------------MemoryExec: partitions=1, partition_sizes=[1] -09)--------CoalesceBatchesExec: target_batch_size=2 -10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +02)--CoalesceBatchesExec: target_batch_size=2 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +04)------CoalesceBatchesExec: target_batch_size=2 +05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------MemoryExec: partitions=1, partition_sizes=[1] +08)------CoalesceBatchesExec: target_batch_size=2 +09)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1353,17 +1352,16 @@ logical_plan physical_plan 01)ProjectionExec: expr=[COUNT(*)@1 as COUNT(*)] 02)--AggregateExec: mode=SinglePartitioned, gby=[t1_id@0 as t1_id], aggr=[COUNT(*)] -03)----ProjectionExec: expr=[t1_id@0 as t1_id] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -06)----------CoalesceBatchesExec: target_batch_size=2 -07)------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -09)----------------MemoryExec: partitions=1, partition_sizes=[1] -10)----------CoalesceBatchesExec: target_batch_size=2 -11)------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -12)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)----------------MemoryExec: partitions=1, partition_sizes=[1] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +05)--------CoalesceBatchesExec: target_batch_size=2 +06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------MemoryExec: partitions=1, partition_sizes=[1] +09)--------CoalesceBatchesExec: target_batch_size=2 +10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT EXPLAIN @@ -1386,17 +1384,16 @@ physical_plan 04)------AggregateExec: mode=Partial, gby=[], aggr=[COUNT(alias1)] 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------AggregateExec: mode=Partial, gby=[t1_id@0 as alias1], aggr=[] -07)------------ProjectionExec: expr=[t1_id@0 as t1_id] -08)--------------CoalesceBatchesExec: target_batch_size=2 -09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] -10)------------------CoalesceBatchesExec: target_batch_size=2 -11)--------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -13)------------------------MemoryExec: partitions=1, partition_sizes=[1] -14)------------------CoalesceBatchesExec: target_batch_size=2 -15)--------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -17)------------------------MemoryExec: partitions=1, partition_sizes=[1] +07)------------CoalesceBatchesExec: target_batch_size=2 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] +09)----------------CoalesceBatchesExec: target_batch_size=2 +10)------------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------MemoryExec: partitions=1, partition_sizes=[1] +13)----------------CoalesceBatchesExec: target_batch_size=2 +14)------------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +15)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)----------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.explain.logical_plan_only = true; @@ -1454,9 +1451,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1480,9 +1477,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@4 as t2_id, t2_name@5 as t2_name, t2_int@6 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)] +01)CoalesceBatchesExec: target_batch_size=2 +02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] @@ -1511,9 +1508,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1537,9 +1534,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] @@ -1571,7 +1568,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] 06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1595,7 +1592,7 @@ logical_plan physical_plan 01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1624,9 +1621,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] +03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] 06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 @@ -1650,9 +1647,9 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name] 04)----TableScan: join_t2 projection=[t2_id] physical_plan -01)ProjectionExec: expr=[t1_id@2 as t1_id, t2_id@0 as t2_id, t1_name@3 as t1_name] +01)ProjectionExec: expr=[t1_id@1 as t1_id, t2_id@0 as t2_id, t1_name@2 as t1_name] 02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)] +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] @@ -1680,13 +1677,12 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] -04)------MemoryExec: partitions=1, partition_sizes=[1] -05)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)----------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +03)----MemoryExec: partitions=1, partition_sizes=[1] +04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok set datafusion.optimizer.repartition_joins = true; @@ -1703,18 +1699,17 @@ logical_plan 02)--TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------CoalesceBatchesExec: target_batch_size=2 -09)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 -10)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 -12)--------------MemoryExec: partitions=1, partition_sizes=[1] +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 +09)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] +10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------MemoryExec: partitions=1, partition_sizes=[1] ##### # Config teardown @@ -2036,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] @@ -3416,13 +3411,12 @@ logical_plan physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] 02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted -03)----ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] -04)------CoalesceBatchesExec: target_batch_size=2 -05)--------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -07)----------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] -08)------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] -09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +03)----CoalesceBatchesExec: target_batch_size=2 +04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +06)--------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +07)----------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] +08)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true # run query above in multiple partitions statement ok diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index b4138f38ea2b..183c2bdc5251 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 ec54e9cdc9f7..ebbebc9ea018 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -635,21 +635,20 @@ logical_plan 05)----Filter: (part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)) AND part.p_size >= Int32(1) 06)------TableScan: part projection=[p_partkey, p_brand, p_size], partial_filters=[part.p_size >= Int32(1), part.p_brand = Utf8("Brand#12") AND part.p_size <= Int32(5) OR part.p_brand = Utf8("Brand#23") AND part.p_size <= Int32(10) OR part.p_brand = Utf8("Brand#34") AND part.p_size <= Int32(15)] physical_plan -01)ProjectionExec: expr=[l_partkey@0 as l_partkey] -02)--CoalesceBatchesExec: target_batch_size=8192 -03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15 -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------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 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true -10)------CoalesceBatchesExec: target_batch_size=8192 -11)--------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -12)----------CoalesceBatchesExec: target_batch_size=8192 -13)------------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 -14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true +01)CoalesceBatchesExec: target_batch_size=8192 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND l_quantity@0 >= Some(100),15,2 AND l_quantity@0 <= Some(1100),15,2 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND l_quantity@0 >= Some(1000),15,2 AND l_quantity@0 <= Some(2000),15,2 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND l_quantity@0 >= Some(2000),15,2 AND l_quantity@0 <= Some(3000),15,2 AND p_size@2 <= 15, projection=[l_partkey@0] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------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 +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true +09)----CoalesceBatchesExec: target_batch_size=8192 +10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +11)--------CoalesceBatchesExec: target_batch_size=8192 +12)----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 +13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true ######## # TPCH Q19 - Pull predicates to inner join (simplified) @@ -728,26 +727,24 @@ logical_plan 10)------TableScan: partsupp projection=[ps_partkey, ps_suppkey] physical_plan 01)AggregateExec: mode=SinglePartitioned, gby=[p_partkey@2 as p_partkey], aggr=[SUM(lineitem.l_extendedprice), AVG(lineitem.l_discount), COUNT(DISTINCT partsupp.ps_suppkey)] -02)--ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, p_partkey@2 as p_partkey, ps_suppkey@4 as ps_suppkey] -03)----CoalesceBatchesExec: target_batch_size=8192 -04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)] -05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_partkey@3 as p_partkey] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, ps_partkey@0)], projection=[l_extendedprice@0, l_discount@1, p_partkey@2, ps_suppkey@4] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true +07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------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 -16)----------------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23 -17)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -18)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true -19)--------CoalesceBatchesExec: target_batch_size=8192 -20)----------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 -21)------------MemoryExec: partitions=1, partition_sizes=[1] +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 +17)------CoalesceBatchesExec: target_batch_size=8192 +18)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 +19)----------MemoryExec: partitions=1, partition_sizes=[1] # Inlist simplification diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 2b298277dd9a..f4c4b5b3599b 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -187,9 +187,9 @@ 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@2 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@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t2_id@0, t1_id@2] +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 @@ -220,9 +220,9 @@ 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@2 as t1_id, SUM(t2.t2_int * Float64(1)) + Int64(1)@0 as t2_sum] -02)--CoalesceBatchesExec: target_batch_size=2 -03)----HashJoinExec: mode=Partitioned, join_type=Right, on=[(t2_id@1, t1_id@0)] +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))] 06)----------CoalesceBatchesExec: target_batch_size=2 @@ -254,9 +254,9 @@ 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 -03)----HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@1)] +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 06)----------MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -291,9 +291,9 @@ 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@2 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@0, t1_id@0)], projection=[SUM(t2.t2_int)@1, t2_id@0, t1_id@2] +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)] diff --git a/datafusion/sqllogictest/test_files/tpch/q1.slt.part b/datafusion/sqllogictest/test_files/tpch/q1.slt.part index 175040420160..8464fee16e1e 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_discountlineitem.l_discountDecimal128(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 d66f5b0f655d..1515040eccd0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q10.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q10.slt.part @@ -78,39 +78,36 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c_custkey@0, c_name@1, c_acctbal@2, c_phone@3, n_name@4, c_address@5, c_comment@6], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_acctbal@4 as c_acctbal, c_phone@3 as c_phone, n_name@8 as n_name, c_address@2 as c_address, c_comment@5 as c_comment], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -09)----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@7 as l_extendedprice, l_discount@8 as l_discount, n_name@10 as n_name] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -14)--------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, l_extendedprice@9 as l_extendedprice, l_discount@10 as l_discount] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@7, l_discount@8, n_name@10] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, l_extendedprice@9, l_discount@10] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@7, l_orderkey@0)] +16)------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@7], 4), input_partitions=4 -19)------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, c_address@2 as c_address, c_nationkey@3 as c_nationkey, c_phone@4 as c_phone, c_acctbal@5 as c_acctbal, c_comment@6 as c_comment, o_orderkey@7 as o_orderkey] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -24)----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -25)------------------------------------------------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 -26)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -28)----------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -29)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------------------------FilterExec: o_orderdate@2 >= 8674 AND o_orderdate@2 < 8766 -31)----------------------------------------------------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 -32)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -34)------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -35)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------FilterExec: l_returnflag@3 = R -37)------------------------------------------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 -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], has_header=false +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, c_address@2, c_nationkey@3, c_phone@4, c_acctbal@5, c_comment@6, o_orderkey@7] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +21)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)----------------------------------------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)--------------------------------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 +36)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +37)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +38)--------------------------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/q11.slt.part b/datafusion/sqllogictest/test_files/tpch/q11.slt.part index 1c2d344daeae..a305f4c82a38 100644 --- a/datafusion/sqllogictest/test_files/tpch/q11.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q11.slt.part @@ -83,54 +83,50 @@ physical_plan 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 09)----------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -10)------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -15)----------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_availqty@2 as ps_availqty, ps_supplycost@3 as ps_supplycost, s_nationkey@5 as s_nationkey] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[ps_partkey@0, ps_availqty@1, ps_supplycost@2] +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)], projection=[ps_partkey@0, ps_availqty@2, ps_supplycost@3, s_nationkey@5] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -20)--------------------------------------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, ps_supplycost], has_header=false -21)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -23)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -24)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -25)------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -27)----------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -28)------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------------FilterExec: n_name@1 = GERMANY -30)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -31)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -32)--------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] -33)----------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -34)------------CoalescePartitionsExec -35)--------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] -36)----------------ProjectionExec: expr=[ps_availqty@0 as ps_availqty, ps_supplycost@1 as ps_supplycost] -37)------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] -39)----------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -41)--------------------------ProjectionExec: expr=[ps_availqty@1 as ps_availqty, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] -42)----------------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)] -44)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)----------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -46)------------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false -47)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -48)----------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -49)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -51)----------------------CoalesceBatchesExec: target_batch_size=8192 -52)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -53)--------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -54)----------------------------CoalesceBatchesExec: target_batch_size=8192 -55)------------------------------FilterExec: n_name@1 = GERMANY -56)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -57)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +17)--------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 +18)----------------------------------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, ps_supplycost], has_header=false +19)------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +21)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------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 +30)--------ProjectionExec: expr=[CAST(CAST(SUM(partsupp.ps_supplycost * partsupp.ps_availqty)@0 AS Float64) * 0.0001 AS Decimal128(38, 15)) as SUM(partsupp.ps_supplycost * partsupp.ps_availqty) * Float64(0.0001)] +31)----------AggregateExec: mode=Final, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +32)------------CoalescePartitionsExec +33)--------------AggregateExec: mode=Partial, gby=[], aggr=[SUM(partsupp.ps_supplycost * partsupp.ps_availqty)] +34)----------------CoalesceBatchesExec: target_batch_size=8192 +35)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[ps_availqty@0, ps_supplycost@1] +36)--------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +38)------------------------CoalesceBatchesExec: target_batch_size=8192 +39)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@0, s_suppkey@0)], projection=[ps_availqty@1, ps_supplycost@2, s_nationkey@4] +40)----------------------------CoalesceBatchesExec: target_batch_size=8192 +41)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +42)--------------------------------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_suppkey, ps_availqty, ps_supplycost], has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +45)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)------------------------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 fd0ebc0cbbd4..a32cac928fe2 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -66,18 +66,17 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([l_shipmode@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[l_shipmode@0 as l_shipmode], aggr=[SUM(CASE WHEN orders.o_orderpriority = Utf8("1-URGENT") OR orders.o_orderpriority = Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END), SUM(CASE WHEN orders.o_orderpriority != Utf8("1-URGENT") AND orders.o_orderpriority != Utf8("2-HIGH") THEN Int64(1) ELSE Int64(0) END)] -08)--------------ProjectionExec: expr=[l_shipmode@1 as l_shipmode, o_orderpriority@3 as o_orderpriority] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_shipmode@1, o_orderpriority@3] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_shipmode@4 as l_shipmode] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------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 -16)------------------------------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 -17)--------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -19)------------------------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_orderpriority], has_header=false +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 +17)--------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +18)----------------------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_orderpriority], has_header=false query TII diff --git a/datafusion/sqllogictest/test_files/tpch/q13.slt.part b/datafusion/sqllogictest/test_files/tpch/q13.slt.part index 0760903470be..e5ea885d9162 100644 --- a/datafusion/sqllogictest/test_files/tpch/q13.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q13.slt.part @@ -64,19 +64,18 @@ physical_plan 08)--------------AggregateExec: mode=Partial, gby=[c_count@0 as c_count], aggr=[COUNT(*)] 09)----------------ProjectionExec: expr=[COUNT(orders.o_orderkey)@1 as c_count] 10)------------------AggregateExec: mode=SinglePartitioned, gby=[c_custkey@0 as c_custkey], aggr=[COUNT(orders.o_orderkey)] -11)--------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, o_orderkey@1 as o_orderkey] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey], has_header=false -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------HashJoinExec: mode=Partitioned, join_type=Left, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, o_orderkey@1] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)----------------------------CoalesceBatchesExec: target_batch_size=8192 20)------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -21)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------------FilterExec: o_comment@2 NOT LIKE %special%requests% -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_orderkey, o_custkey, o_comment], has_header=false +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 b6ca92129cd6..d53fe0b3177e 100644 --- a/datafusion/sqllogictest/test_files/tpch/q14.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q14.slt.part @@ -45,19 +45,18 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(CASE WHEN part.p_type LIKE Utf8("PROMO%") THEN lineitem.l_extendedprice * Int64(1) - lineitem.l_discount ELSE Int64(0) END), SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, p_type@4 as p_type] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +05)--------CoalesceBatchesExec: target_batch_size=8192 +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)----------------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)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------FilterExec: l_shipdate@3 >= 9374 AND l_shipdate@3 < 9404 -13)------------------------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 -14)--------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -17)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false +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 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +16)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q15.slt.part b/datafusion/sqllogictest/test_files/tpch/q15.slt.part index 94693ca31e2c..51d45ba5ff4c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q15.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q15.slt.part @@ -73,41 +73,39 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_suppkey@0 ASC NULLS LAST] 02)--SortExec: expr=[s_suppkey@0 ASC NULLS LAST] -03)----ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@4 as total_revenue] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 -08)--------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address, s_phone@3 as s_phone, total_revenue@5 as total_revenue] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(total_revenue@4, MAX(revenue0.total_revenue)@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@4] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([total_revenue@4], 4), input_partitions=4 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)], projection=[s_suppkey@0, s_name@1, s_address@2, s_phone@3, total_revenue@5] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, supplier_no@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false -15)--------------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -16)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -17)------------------------CoalesceBatchesExec: target_batch_size=8192 -18)--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -19)----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -20)------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -21)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -23)------------------------------------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 -24)----------CoalesceBatchesExec: target_batch_size=8192 -25)------------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 -26)--------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] -27)----------------CoalescePartitionsExec -28)------------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] -29)--------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] -30)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -31)------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------RepartitionExec: partitioning=Hash([l_suppkey@0], 4), input_partitions=4 -33)----------------------------AggregateExec: mode=Partial, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -34)------------------------------ProjectionExec: expr=[l_suppkey@0 as l_suppkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount] -35)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------FilterExec: l_shipdate@3 >= 9496 AND l_shipdate@3 < 9587 -37)------------------------------------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 +10)------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_phone], has_header=false +13)----------------ProjectionExec: expr=[l_suppkey@0 as supplier_no, SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +14)------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +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)--------------------------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 +23)----------RepartitionExec: partitioning=Hash([MAX(revenue0.total_revenue)@0], 4), input_partitions=1 +24)------------AggregateExec: mode=Final, gby=[], aggr=[MAX(revenue0.total_revenue)] +25)--------------CoalescePartitionsExec +26)----------------AggregateExec: mode=Partial, gby=[], aggr=[MAX(revenue0.total_revenue)] +27)------------------ProjectionExec: expr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)@1 as total_revenue] +28)--------------------AggregateExec: mode=FinalPartitioned, gby=[l_suppkey@0 as l_suppkey], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] +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)----------------------------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 query ITTTR with revenue0 (supplier_no, total_revenue) as ( diff --git a/datafusion/sqllogictest/test_files/tpch/q16.slt.part b/datafusion/sqllogictest/test_files/tpch/q16.slt.part index 589b161c4569..8c679421ce50 100644 --- a/datafusion/sqllogictest/test_files/tpch/q16.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q16.slt.part @@ -82,25 +82,24 @@ physical_plan 14)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(ps_suppkey@0, s_suppkey@0)] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 16)------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -17)--------------------------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey, p_brand@3 as p_brand, p_type@4 as p_type, p_size@5 as p_size] -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -22)------------------------------------------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], has_header=false -23)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -25)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) -27)----------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -28)------------------------------------------------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 -29)----------------------------CoalesceBatchesExec: target_batch_size=8192 -30)------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -31)--------------------------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey] -32)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)------------------------------------FilterExec: s_comment@1 LIKE %Customer%Complaints% -34)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -35)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_comment], has_header=false +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, p_partkey@0)], projection=[ps_suppkey@1, p_brand@3, p_type@4, p_size@5] +19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +20)--------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +21)----------------------------------------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], has_header=false +22)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +24)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------FilterExec: p_brand@1 != Brand#45 AND p_type@2 NOT LIKE MEDIUM POLISHED% AND Use p_size@3 IN (SET) ([Literal { value: Int32(49) }, Literal { value: Int32(14) }, Literal { value: Int32(23) }, Literal { value: Int32(45) }, Literal { value: Int32(19) }, Literal { value: Int32(3) }, Literal { value: Int32(36) }, Literal { value: Int32(9) }]) +26)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------------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 3dca369b20ac..4ca2eee20910 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -55,28 +55,26 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice)] -05)--------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 -08)--------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1, projection=[l_extendedprice@1] +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_quantity@1, l_extendedprice@2, p_partkey@3] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -13)------------------------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 +10)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +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)--------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -17)--------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -19)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -20)--------------------------------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 -21)--------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] -22)----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -23)------------------CoalesceBatchesExec: target_batch_size=8192 -24)--------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -25)----------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] -26)------------------------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], has_header=false +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 +19)------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] +20)--------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +21)----------------CoalesceBatchesExec: target_batch_size=8192 +22)------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +23)--------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] +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_partkey, l_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q18.slt.part b/datafusion/sqllogictest/test_files/tpch/q18.slt.part index 2b771156f83c..7917a1b065e3 100644 --- a/datafusion/sqllogictest/test_files/tpch/q18.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q18.slt.part @@ -75,32 +75,30 @@ physical_plan 06)----------AggregateExec: mode=Partial, gby=[c_name@1 as c_name, c_custkey@0 as c_custkey, o_orderkey@2 as o_orderkey, o_orderdate@4 as o_orderdate, o_totalprice@3 as o_totalprice], aggr=[SUM(lineitem.l_quantity)] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@2, l_orderkey@0)] -09)----------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@3 as o_totalprice, o_orderdate@4 as o_orderdate, l_quantity@6 as l_quantity] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 -14)--------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey, c_name@1 as c_name, o_orderkey@2 as o_orderkey, o_totalprice@4 as o_totalprice, o_orderdate@5 as o_orderdate] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@2, l_orderkey@0)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@3, o_orderdate@4, l_quantity@6] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@2], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_custkey@0, c_name@1, o_orderkey@2, o_totalprice@4, o_orderdate@5] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -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_name], has_header=false -21)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 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_orderkey, o_custkey, o_totalprice, o_orderdate], has_header=false -24)----------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -26)--------------------------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 -27)----------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -28)------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------FilterExec: SUM(lineitem.l_quantity)@1 > Some(30000),25,2 -30)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -31)------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -33)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] -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_quantity], has_header=false +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +17)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +18)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_name], has_header=false +19)----------------------------CoalesceBatchesExec: target_batch_size=8192 +20)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 +21)--------------------------------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_totalprice, o_orderdate], has_header=false +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)----------------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 +30)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +31)----------------------------AggregateExec: mode=Partial, gby=[l_orderkey@0 as l_orderkey], aggr=[SUM(lineitem.l_quantity)] +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_quantity], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q19.slt.part b/datafusion/sqllogictest/test_files/tpch/q19.slt.part index 383d532d9406..51150d6f2f50 100644 --- a/datafusion/sqllogictest/test_files/tpch/q19.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q19.slt.part @@ -68,21 +68,20 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -05)--------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------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 -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], filter=p_brand@1 = Brand#12 AND 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)----------------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)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------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 -13)------------------------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 -14)--------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -16)------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 -18)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -19)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_size, p_container], has_header=false +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 +14)--------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +15)----------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------FilterExec: (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 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 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 p_size@2 <= 15) AND p_size@2 >= 1 +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_size, p_container], has_header=false diff --git a/datafusion/sqllogictest/test_files/tpch/q2.slt.part b/datafusion/sqllogictest/test_files/tpch/q2.slt.part index ee6341d5960c..c7c8e546f136 100644 --- a/datafusion/sqllogictest/test_files/tpch/q2.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q2.slt.part @@ -103,91 +103,84 @@ physical_plan 01)GlobalLimitExec: skip=0, fetch=10 02)--SortPreservingMergeExec: [s_acctbal@0 DESC,n_name@2 ASC NULLS LAST,s_name@1 ASC NULLS LAST,p_partkey@3 ASC NULLS LAST], fetch=10 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] -04)------ProjectionExec: expr=[s_acctbal@5 as s_acctbal, s_name@2 as s_name, n_name@8 as n_name, p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_address@3 as s_address, s_phone@4 as s_phone, s_comment@6 as s_comment] +04)------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@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, ps_supplycost@7, n_name@8, MIN(partsupp.ps_supplycost)@10, ps_partkey@9] +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)----------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@4 as s_phone, s_acctbal@5 as s_acctbal, s_comment@6 as s_comment, ps_supplycost@7 as ps_supplycost, n_name@8 as n_name] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([n_regionkey@9], 4), input_partitions=4 -14)--------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@2 as s_name, s_address@3 as s_address, s_phone@5 as s_phone, s_acctbal@6 as s_acctbal, s_comment@7 as s_comment, ps_supplycost@8 as ps_supplycost, n_name@10 as n_name, n_regionkey@11 as n_regionkey] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@9, r_regionkey@0)], projection=[p_partkey@0, p_mfgr@1, s_name@2, s_address@3, s_phone@4, s_acctbal@5, s_comment@6, ps_supplycost@7, n_name@8] +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@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)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@4, n_nationkey@0)] +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@5], 4), input_partitions=4 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------RepartitionExec: partitioning=Hash([s_nationkey@4], 4), input_partitions=4 -19)------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, s_name@5 as s_name, s_address@6 as s_address, s_nationkey@7 as s_nationkey, s_phone@8 as s_phone, s_acctbal@9 as s_acctbal, s_comment@10 as s_comment, ps_supplycost@3 as ps_supplycost] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@2, s_suppkey@0)] -22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@2], 4), input_partitions=4 -24)----------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr, ps_suppkey@3 as ps_suppkey, ps_supplycost@4 as ps_supplycost] +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)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, ps_partkey@0)] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -29)--------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] -30)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)------------------------------------------------------------FilterExec: p_size@3 = 15 AND p_type@2 LIKE %BRASS -32)--------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -33)----------------------------------------------------------------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 -34)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)------------------------------------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -36)--------------------------------------------------------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 -37)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)--------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@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/supplier.tbl]]}, projection=[s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment], has_header=false -41)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -43)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -44)--------------------------------------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 -45)----------------------CoalesceBatchesExec: target_batch_size=8192 -46)------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -47)--------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -48)----------------------------CoalesceBatchesExec: target_batch_size=8192 -49)------------------------------FilterExec: r_name@1 = EUROPE -50)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)----------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false -52)------------CoalesceBatchesExec: target_batch_size=8192 -53)--------------RepartitionExec: partitioning=Hash([ps_partkey@0, MIN(partsupp.ps_supplycost)@1], 4), input_partitions=4 -54)----------------AggregateExec: mode=FinalPartitioned, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] -55)------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 -57)----------------------AggregateExec: mode=Partial, gby=[ps_partkey@0 as ps_partkey], aggr=[MIN(partsupp.ps_supplycost)] -58)------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost] -59)--------------------------CoalesceBatchesExec: target_batch_size=8192 -60)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@2, r_regionkey@0)] -61)------------------------------CoalesceBatchesExec: target_batch_size=8192 -62)--------------------------------RepartitionExec: partitioning=Hash([n_regionkey@2], 4), input_partitions=4 -63)----------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@1 as ps_supplycost, n_regionkey@4 as n_regionkey] -64)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -65)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] -66)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -67)------------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -68)--------------------------------------------ProjectionExec: expr=[ps_partkey@0 as ps_partkey, ps_supplycost@2 as ps_supplycost, s_nationkey@4 as s_nationkey] -69)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -70)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_suppkey@1, s_suppkey@0)] -71)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -72)----------------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1], 4), input_partitions=4 -73)------------------------------------------------------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 -74)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -75)----------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -76)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -77)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -78)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -79)------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -80)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -81)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -82)------------------------------CoalesceBatchesExec: target_batch_size=8192 -83)--------------------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -84)----------------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -85)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -86)--------------------------------------FilterExec: r_name@1 = EUROPE -87)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -88)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +26)--------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey, p_mfgr@1 as p_mfgr] +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 af3f538f7a03..8e081ac4e94f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q20.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q20.slt.part @@ -83,53 +83,50 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [s_name@0 ASC NULLS LAST] 02)--SortExec: expr=[s_name@0 ASC NULLS LAST] -03)----ProjectionExec: expr=[s_name@1 as s_name, s_address@2 as s_address] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)] -06)----------CoalesceBatchesExec: target_batch_size=8192 -07)------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -08)--------------ProjectionExec: expr=[s_suppkey@0 as s_suppkey, s_name@1 as s_name, s_address@2 as s_address] +03)----CoalesceBatchesExec: target_batch_size=8192 +04)------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(s_suppkey@0, ps_suppkey@0)], projection=[s_name@1, s_address@2] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +07)------------CoalesceBatchesExec: target_batch_size=8192 +08)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[s_suppkey@0, s_name@1, s_address@2] 09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -13)------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -14)--------------------------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 +10)------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 +11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------CoalesceBatchesExec: target_batch_size=8192 -16)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -17)------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -18)--------------------------CoalesceBatchesExec: target_batch_size=8192 -19)----------------------------FilterExec: n_name@1 = CANADA -20)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -21)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -22)----------CoalesceBatchesExec: target_batch_size=8192 -23)------------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 -24)--------------ProjectionExec: expr=[ps_suppkey@1 as ps_suppkey] -25)----------------CoalesceBatchesExec: target_batch_size=8192 -26)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1 -27)--------------------CoalesceBatchesExec: target_batch_size=8192 -28)----------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 -29)------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] -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_availqty], has_header=false -34)----------------------------CoalesceBatchesExec: target_batch_size=8192 -35)------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -36)--------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -37)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------------------FilterExec: p_name@1 LIKE forest% -39)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -40)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -41)--------------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] -42)----------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -43)------------------------CoalesceBatchesExec: target_batch_size=8192 -44)--------------------------RepartitionExec: partitioning=Hash([l_partkey@0, l_suppkey@1], 4), input_partitions=4 -45)----------------------------AggregateExec: mode=Partial, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] -46)------------------------------ProjectionExec: expr=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey, l_quantity@2 as l_quantity] -47)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -48)----------------------------------FilterExec: l_shipdate@3 >= 8766 AND l_shipdate@3 < 9131 -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_partkey, l_suppkey, l_quantity, l_shipdate], has_header=false +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 +20)--------CoalesceBatchesExec: target_batch_size=8192 +21)----------RepartitionExec: partitioning=Hash([ps_suppkey@0], 4), input_partitions=4 +22)------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(ps_partkey@0, l_partkey@1), (ps_suppkey@1, l_suppkey@2)], filter=CAST(ps_availqty@0 AS Float64) > Float64(0.5) * SUM(lineitem.l_quantity)@1, projection=[ps_suppkey@1] +24)----------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------RepartitionExec: partitioning=Hash([ps_partkey@0, ps_suppkey@1], 4), input_partitions=4 +26)--------------------CoalesceBatchesExec: target_batch_size=8192 +27)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(ps_partkey@0, p_partkey@0)] +28)------------------------CoalesceBatchesExec: target_batch_size=8192 +29)--------------------------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=4 +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)----------------------------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 +38)----------------ProjectionExec: expr=[0.5 * CAST(SUM(lineitem.l_quantity)@2 AS Float64) as Float64(0.5) * SUM(lineitem.l_quantity), l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey] +39)------------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey, l_suppkey@1 as l_suppkey], aggr=[SUM(lineitem.l_quantity)] +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)--------------------------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 959d96f8a693..3d6428882374 100644 --- a/datafusion/sqllogictest/test_files/tpch/q21.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q21.slt.part @@ -97,58 +97,54 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([s_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[s_name@0 as s_name], aggr=[COUNT(*)] -08)--------------ProjectionExec: expr=[s_name@0 as s_name] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 -13)------------------------CoalesceBatchesExec: target_batch_size=8192 -14)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -15)----------------------------ProjectionExec: expr=[s_name@0 as s_name, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0, projection=[s_name@0] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(l_orderkey@1, l_orderkey@0)], filter=l_suppkey@1 != l_suppkey@0 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)], projection=[s_name@0, l_orderkey@2, l_suppkey@3] 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@1, n_nationkey@0)] +17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------RepartitionExec: partitioning=Hash([s_nationkey@1], 4), input_partitions=4 -20)--------------------------------------ProjectionExec: expr=[s_name@0 as s_name, s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_suppkey@3 as l_suppkey] -21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)] -23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 -25)------------------------------------------------ProjectionExec: expr=[s_name@1 as s_name, s_nationkey@2 as s_nationkey, l_orderkey@3 as l_orderkey, l_suppkey@4 as l_suppkey] -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] -28)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -29)--------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -30)----------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -31)------------------------------------------------------------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 -32)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -34)----------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -35)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -37)----------------------------------------------------------------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 -38)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -40)------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey] -41)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------------------------FilterExec: o_orderstatus@1 = F -43)------------------------------------------------------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 -44)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -46)--------------------------------------ProjectionExec: expr=[n_nationkey@0 as n_nationkey] -47)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -48)------------------------------------------FilterExec: n_name@1 = SAUDI ARABIA -49)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -50)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -51)------------------------CoalesceBatchesExec: target_batch_size=8192 -52)--------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -53)----------------------------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 -54)--------------------CoalesceBatchesExec: target_batch_size=8192 -55)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -56)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_suppkey@1 as l_suppkey] -57)--------------------------CoalesceBatchesExec: target_batch_size=8192 -58)----------------------------FilterExec: l_receiptdate@3 > l_commitdate@2 -59)------------------------------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 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@2, o_orderkey@0)], projection=[s_name@0, s_nationkey@1, l_orderkey@2, l_suppkey@3] +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@2], 4), input_partitions=4 +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_name@1, s_nationkey@2, l_orderkey@3, l_suppkey@4] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +26)--------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)--------------------------------------------------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)------------------------------------------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)----------------------------------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 +47)----------------------CoalesceBatchesExec: target_batch_size=8192 +48)------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +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)----------------------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 5faaf8a73d6a..58095ba33652 100644 --- a/datafusion/sqllogictest/test_files/tpch/q22.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q22.slt.part @@ -86,26 +86,25 @@ physical_plan 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)------------------------ProjectionExec: expr=[c_phone@1 as c_phone, c_acctbal@2 as c_acctbal] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(c_custkey@0, o_custkey@0)] -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -19)------------------------------------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") }]) -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_phone, c_acctbal], has_header=false -22)------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------RepartitionExec: partitioning=Hash([o_custkey@0], 4), input_partitions=4 -24)----------------------------------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 -25)----------------------AggregateExec: mode=Final, gby=[], aggr=[AVG(customer.c_acctbal)] -26)------------------------CoalescePartitionsExec -27)--------------------------AggregateExec: mode=Partial, gby=[], aggr=[AVG(customer.c_acctbal)] -28)----------------------------ProjectionExec: expr=[c_acctbal@1 as c_acctbal] -29)------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)--------------------------------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") }]) -31)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -32)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_phone, c_acctbal], has_header=false +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)------------------------------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 b2689c6a319f..e8d352090a24 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -67,32 +67,30 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([l_orderkey@0, o_orderdate@1, o_shippriority@2], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[l_orderkey@2 as l_orderkey, o_orderdate@0 as o_orderdate, o_shippriority@1 as o_shippriority], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -09)----------------ProjectionExec: expr=[o_orderdate@1 as o_orderdate, o_shippriority@2 as o_shippriority, l_orderkey@3 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)] -12)----------------------CoalesceBatchesExec: target_batch_size=8192 -13)------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -14)--------------------------ProjectionExec: expr=[o_orderkey@1 as o_orderkey, o_orderdate@3 as o_orderdate, o_shippriority@4 as o_shippriority] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@0, l_orderkey@0)], projection=[o_orderdate@1, o_shippriority@2, l_orderkey@3, l_extendedprice@4, l_discount@5] +11)--------------------CoalesceBatchesExec: target_batch_size=8192 +12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[o_orderkey@1, o_orderdate@3, o_shippriority@4] 15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] +16)------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -19)------------------------------------ProjectionExec: expr=[c_custkey@0 as c_custkey] -20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------FilterExec: c_mktsegment@1 = BUILDING -22)------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -23)--------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_mktsegment], has_header=false +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 +22)----------------------------CoalesceBatchesExec: target_batch_size=8192 +23)------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 24)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)----------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -26)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)--------------------------------------FilterExec: o_orderdate@2 < 9204 -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, o_shippriority], 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 -33)------------------------------FilterExec: l_shipdate@3 > 9204 -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_shipdate], has_header=false +25)----------------------------------FilterExec: o_orderdate@2 < 9204 +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)------------------------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 874d5abc201b..929b5cbefa8c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q4.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q4.slt.part @@ -60,21 +60,20 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_orderpriority@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_orderpriority@0 as o_orderpriority], aggr=[COUNT(*)] -08)--------------ProjectionExec: expr=[o_orderpriority@1 as o_orderpriority] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(o_orderkey@0, l_orderkey@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +08)--------------CoalesceBatchesExec: target_batch_size=8192 +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)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_orderpriority@2 as o_orderpriority] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------FilterExec: o_orderdate@1 >= 8582 AND o_orderdate@1 < 8674 -16)------------------------------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 -17)--------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +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)----------------------CoalesceBatchesExec: target_batch_size=8192 19)------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey] -20)--------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------FilterExec: l_receiptdate@2 > l_commitdate@1 -22)------------------------------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 +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 4b4b23e821eb..7744a2b6038d 100644 --- a/datafusion/sqllogictest/test_files/tpch/q5.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q5.slt.part @@ -74,57 +74,52 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([n_name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[n_name@2 as n_name], aggr=[SUM(lineitem.l_extendedprice * Int64(1) - lineitem.l_discount)] -08)--------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@2 as n_name] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] -11)--------------------CoalesceBatchesExec: target_batch_size=8192 -12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -13)------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, n_name@4 as n_name, n_regionkey@5 as n_regionkey] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@2] +10)------------------CoalesceBatchesExec: target_batch_size=8192 +11)--------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 +12)----------------------CoalesceBatchesExec: target_batch_size=8192 +13)------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, n_name@4, n_regionkey@5] 14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] +15)----------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -18)----------------------------------ProjectionExec: expr=[l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] -19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)] -21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 -23)--------------------------------------------ProjectionExec: expr=[c_nationkey@0 as c_nationkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] +17)--------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0), (c_nationkey@0, s_nationkey@1)], projection=[l_extendedprice@2, l_discount@3, s_nationkey@5] +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1, c_nationkey@0], 4), input_partitions=4 +20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)], projection=[c_nationkey@0, l_suppkey@3, l_extendedprice@4, l_discount@5] +22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +23)--------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_orderkey@1, l_orderkey@0)] +25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)], projection=[c_nationkey@1, o_orderkey@2] 26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@1], 4), input_partitions=4 -28)------------------------------------------------------ProjectionExec: expr=[c_nationkey@1 as c_nationkey, o_orderkey@2 as o_orderkey] -29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_custkey@0, o_custkey@1)] -31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -33)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -34)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -35)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@1], 4), input_partitions=4 -37)----------------------------------------------------------------ProjectionExec: expr=[o_orderkey@0 as o_orderkey, o_custkey@1 as o_custkey] -38)------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -39)--------------------------------------------------------------------FilterExec: o_orderdate@2 >= 8766 AND o_orderdate@2 < 9131 -40)----------------------------------------------------------------------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 -41)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -43)------------------------------------------------------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_extendedprice, l_discount], has_header=false -44)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 -46)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -48)------------------------------CoalesceBatchesExec: target_batch_size=8192 -49)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -50)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -51)------------------------------------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 -52)--------------------CoalesceBatchesExec: target_batch_size=8192 -53)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -54)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -55)--------------------------CoalesceBatchesExec: target_batch_size=8192 -56)----------------------------FilterExec: r_name@1 = ASIA -57)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -58)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +27)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +28)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)------------------------------------------------------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 +37)--------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +38)----------------------------------------------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_extendedprice, l_discount], has_header=false +39)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0, s_nationkey@1], 4), input_partitions=4 +41)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +43)--------------------------CoalesceBatchesExec: target_batch_size=8192 +44)----------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +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, n_regionkey], has_header=false +47)------------------CoalesceBatchesExec: target_batch_size=8192 +48)--------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 +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/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 25f42304be4b..99d5c934c9ad 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -91,57 +91,53 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] -08)--------------ProjectionExec: expr=[n_name@4 as supp_nation, n_name@6 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 -13)------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@4 as c_nationkey, n_name@6 as n_name] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)] -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 -18)----------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, l_shipdate@3 as l_shipdate, c_nationkey@6 as c_nationkey] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@0, n_nationkey@0)], projection=[l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@4, n_name@6] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@0], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)], projection=[s_nationkey@0, l_extendedprice@1, l_discount@2, l_shipdate@3, c_nationkey@6] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@4, c_custkey@0)] +20)--------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@4], 4), input_partitions=4 -23)--------------------------------------------ProjectionExec: expr=[s_nationkey@0 as s_nationkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, l_shipdate@4 as l_shipdate, o_custkey@6 as o_custkey] -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)] -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 -28)------------------------------------------------------ProjectionExec: expr=[s_nationkey@1 as s_nationkey, l_orderkey@2 as l_orderkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, l_shipdate@6 as l_shipdate] -29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)] -31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -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_nationkey], has_header=false -35)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -37)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -38)------------------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 -39)--------------------------------------------------------------------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_extendedprice, l_discount, l_shipdate], has_header=false -40)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -41)----------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -42)------------------------------------------------------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], has_header=false -43)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -44)------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -45)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -46)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -47)------------------------------CoalesceBatchesExec: target_batch_size=8192 -48)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -49)----------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)------------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY -51)--------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -52)----------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -53)--------------------CoalesceBatchesExec: target_batch_size=8192 -54)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -55)------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE -57)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -58)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@1, o_orderkey@0)], projection=[s_nationkey@0, l_extendedprice@2, l_discount@3, l_shipdate@4, o_custkey@6] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@1], 4), input_partitions=4 +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_suppkey@0, l_suppkey@1)], projection=[s_nationkey@1, l_orderkey@2, l_extendedprice@4, l_discount@5, l_shipdate@6] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +29)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +31)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +32)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +33)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)----------------------------------------------------------FilterExec: l_shipdate@4 >= 9131 AND l_shipdate@4 <= 9861 +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_orderkey, l_suppkey, l_extendedprice, l_discount, l_shipdate], has_header=false +36)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +37)----------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +38)------------------------------------------------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], has_header=false +39)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +40)--------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +41)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +42)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +43)----------------------------CoalesceBatchesExec: target_batch_size=8192 +44)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +45)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)----------------------------------FilterExec: n_name@1 = FRANCE OR n_name@1 = GERMANY +47)------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)--------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +49)--------------------CoalesceBatchesExec: target_batch_size=8192 +50)----------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +51)------------------------CoalesceBatchesExec: target_batch_size=8192 +52)--------------------------FilterExec: n_name@1 = GERMANY OR n_name@1 = FRANCE +53)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +54)------------------------------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/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 69e778646784..c8d67d0d4905 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -97,77 +97,71 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] -08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@4 as nation] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)] +08)--------------CoalesceBatchesExec: target_batch_size=8192 +09)----------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] +10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 -13)------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, o_orderdate@3 as o_orderdate, n_regionkey@4 as n_regionkey, n_name@6 as n_name] -14)--------------------------CoalesceBatchesExec: target_batch_size=8192 -15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)] -16)------------------------------CoalesceBatchesExec: target_batch_size=8192 -17)--------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 -18)----------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@3 as o_orderdate, n_regionkey@6 as n_regionkey] +13)------------------------CoalesceBatchesExec: target_batch_size=8192 +14)--------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@2, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@3, n_regionkey@4, n_name@6] +15)----------------------------CoalesceBatchesExec: target_batch_size=8192 +16)------------------------------RepartitionExec: partitioning=Hash([s_nationkey@2], 4), input_partitions=4 +17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 +18)----------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@3, n_regionkey@6] 19)------------------------------------CoalesceBatchesExec: target_batch_size=8192 -20)--------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@4, n_nationkey@0)] +20)--------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 21)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -22)------------------------------------------RepartitionExec: partitioning=Hash([c_nationkey@4], 4), input_partitions=4 -23)--------------------------------------------ProjectionExec: expr=[l_extendedprice@0 as l_extendedprice, l_discount@1 as l_discount, s_nationkey@2 as s_nationkey, o_orderdate@4 as o_orderdate, c_nationkey@6 as c_nationkey] -24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -25)------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)] -26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -27)----------------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 -28)------------------------------------------------------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice, l_discount@2 as l_discount, s_nationkey@3 as s_nationkey, o_custkey@5 as o_custkey, o_orderdate@6 as o_orderdate] +22)------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(o_custkey@3, c_custkey@0)], projection=[l_extendedprice@0, l_discount@1, s_nationkey@2, o_orderdate@4, c_nationkey@6] +23)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +24)----------------------------------------------RepartitionExec: partitioning=Hash([o_custkey@3], 4), input_partitions=4 +25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_extendedprice@1, l_discount@2, s_nationkey@3, o_custkey@5, o_orderdate@6] +27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 29)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] +30)----------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)], projection=[l_orderkey@0, l_extendedprice@2, l_discount@3, s_nationkey@5] 31)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -33)----------------------------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@5 as s_nationkey] -34)------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -35)--------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@1, s_suppkey@0)] -36)----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -37)------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 -38)--------------------------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_suppkey@3 as l_suppkey, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount] -39)----------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)------------------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] -41)--------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -42)----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -43)------------------------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -44)--------------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -45)----------------------------------------------------------------------------------------FilterExec: p_type@1 = ECONOMY ANODIZED STEEL -46)------------------------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -47)--------------------------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_type], has_header=false -48)--------------------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -49)----------------------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -50)------------------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false -51)----------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -52)------------------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -53)--------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -54)----------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -55)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -56)--------------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -57)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -58)------------------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 -59)--------------------------------------------------------------------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 -60)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -61)----------------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 -62)------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -63)--------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false -64)----------------------------------------CoalesceBatchesExec: target_batch_size=8192 -65)------------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -66)--------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -67)----------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false -68)------------------------------CoalesceBatchesExec: target_batch_size=8192 -69)--------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -70)----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -71)------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false -72)--------------------CoalesceBatchesExec: target_batch_size=8192 -73)----------------------RepartitionExec: partitioning=Hash([r_regionkey@0], 4), input_partitions=4 -74)------------------------ProjectionExec: expr=[r_regionkey@0 as r_regionkey] -75)--------------------------CoalesceBatchesExec: target_batch_size=8192 -76)----------------------------FilterExec: r_name@1 = AMERICA -77)------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -78)--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/region.tbl]]}, projection=[r_regionkey, r_name], has_header=false +32)--------------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@1], 4), input_partitions=4 +33)----------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +34)------------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)], projection=[l_orderkey@1, l_suppkey@3, l_extendedprice@4, l_discount@5] +35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +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 +42)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +44)------------------------------------------------------------------------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_partkey, l_suppkey, l_extendedprice, l_discount], has_header=false +45)------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 +47)----------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +48)------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +49)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +50)------------------------------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +51)--------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +52)----------------------------------------------------------FilterExec: o_orderdate@2 >= 9131 AND o_orderdate@2 <= 9861 +53)------------------------------------------------------------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 +54)--------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +55)----------------------------------------------RepartitionExec: partitioning=Hash([c_custkey@0], 4), input_partitions=4 +56)------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +57)--------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/customer.tbl]]}, projection=[c_custkey, c_nationkey], has_header=false +58)------------------------------------CoalesceBatchesExec: target_batch_size=8192 +59)--------------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +60)----------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +61)------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_regionkey], has_header=false +62)----------------------------CoalesceBatchesExec: target_batch_size=8192 +63)------------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +64)--------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +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)------------------------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 6a59294526df..82480464370c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -84,53 +84,49 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -09)----------------ProjectionExec: expr=[n_name@7 as nation, date_part(YEAR, o_orderdate@5) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@4 * l_quantity@0 as amount] -10)------------------CoalesceBatchesExec: target_batch_size=8192 -11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)] +09)----------------CoalesceBatchesExec: target_batch_size=8192 +10)------------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] +11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 -14)--------------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, l_discount@3 as l_discount, s_nationkey@4 as s_nationkey, ps_supplycost@5 as ps_supplycost, o_orderdate@7 as o_orderdate] -15)----------------------------CoalesceBatchesExec: target_batch_size=8192 -16)------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)] -17)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -18)----------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 -19)------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@6 as s_nationkey, ps_supplycost@9 as ps_supplycost] +14)--------------------------CoalesceBatchesExec: target_batch_size=8192 +15)----------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_orderkey@0, o_orderkey@0)], projection=[l_quantity@1, l_extendedprice@2, l_discount@3, s_nationkey@4, ps_supplycost@5, o_orderdate@7] +16)------------------------------CoalesceBatchesExec: target_batch_size=8192 +17)--------------------------------RepartitionExec: partitioning=Hash([l_orderkey@0], 4), input_partitions=4 +18)----------------------------------CoalesceBatchesExec: target_batch_size=8192 +19)------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)], projection=[l_orderkey@0, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@6, ps_supplycost@9] 20)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 -21)----------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, ps_suppkey@1), (l_partkey@1, ps_partkey@0)] +21)----------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 22)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -23)--------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2, l_partkey@1], 4), input_partitions=4 -24)----------------------------------------------ProjectionExec: expr=[l_orderkey@0 as l_orderkey, l_partkey@1 as l_partkey, l_suppkey@2 as l_suppkey, l_quantity@3 as l_quantity, l_extendedprice@4 as l_extendedprice, l_discount@5 as l_discount, s_nationkey@7 as s_nationkey] -25)------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -26)--------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)] -27)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -28)------------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 -29)--------------------------------------------------------ProjectionExec: expr=[l_orderkey@1 as l_orderkey, l_partkey@2 as l_partkey, l_suppkey@3 as l_suppkey, l_quantity@4 as l_quantity, l_extendedprice@5 as l_extendedprice, l_discount@6 as l_discount] +23)--------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_suppkey@2, s_suppkey@0)], projection=[l_orderkey@0, l_partkey@1, l_suppkey@2, l_quantity@3, l_extendedprice@4, l_discount@5, s_nationkey@7] +24)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +25)------------------------------------------------RepartitionExec: partitioning=Hash([l_suppkey@2], 4), input_partitions=4 +26)--------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +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)----------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -31)------------------------------------------------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@0, l_partkey@1)] -32)--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -33)----------------------------------------------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -34)------------------------------------------------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -35)--------------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -36)----------------------------------------------------------------------FilterExec: p_name@1 LIKE %green% -37)------------------------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -38)--------------------------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_name], has_header=false -39)--------------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -40)----------------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 -41)------------------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false -42)----------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -43)------------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@0], 4), input_partitions=4 -44)--------------------------------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -45)----------------------------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false -46)------------------------------------------CoalesceBatchesExec: target_batch_size=8192 -47)--------------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 -48)----------------------------------------------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 -49)--------------------------------CoalesceBatchesExec: target_batch_size=8192 -50)----------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 -51)------------------------------------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], has_header=false -52)----------------------CoalesceBatchesExec: target_batch_size=8192 -53)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 -54)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -55)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false +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 +35)------------------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +36)--------------------------------------------------------RepartitionExec: partitioning=Hash([l_partkey@1], 4), input_partitions=4 +37)----------------------------------------------------------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_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount], has_header=false +38)----------------------------------------------CoalesceBatchesExec: target_batch_size=8192 +39)------------------------------------------------RepartitionExec: partitioning=Hash([s_suppkey@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/supplier.tbl]]}, projection=[s_suppkey, s_nationkey], has_header=false +42)--------------------------------------CoalesceBatchesExec: target_batch_size=8192 +43)----------------------------------------RepartitionExec: partitioning=Hash([ps_suppkey@1, ps_partkey@0], 4), input_partitions=4 +44)------------------------------------------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 +45)------------------------------CoalesceBatchesExec: target_batch_size=8192 +46)--------------------------------RepartitionExec: partitioning=Hash([o_orderkey@0], 4), input_partitions=4 +47)----------------------------------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], has_header=false +48)----------------------CoalesceBatchesExec: target_batch_size=8192 +49)------------------------RepartitionExec: partitioning=Hash([n_nationkey@0], 4), input_partitions=4 +50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +51)----------------------------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/union.slt b/datafusion/sqllogictest/test_files/union.slt index 8ae7c5b34135..6b0d53fb124f 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -285,28 +285,27 @@ physical_plan 07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] -10)------ProjectionExec: expr=[name@1 as name, CAST(t2.id AS Int32)@2 as CAST(t2.id AS Int32)] -11)--------CoalesceBatchesExec: target_batch_size=2 -12)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -13)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -15)----------------MemoryExec: partitions=1, partition_sizes=[1] -16)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] -17)----CoalesceBatchesExec: target_batch_size=2 -18)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)] -19)--------CoalesceBatchesExec: target_batch_size=2 -20)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 -21)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -22)--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] -23)----------------CoalesceBatchesExec: target_batch_size=2 -24)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -25)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -26)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -27)------------------------MemoryExec: partitions=1, partition_sizes=[1] -28)--------CoalesceBatchesExec: target_batch_size=2 -29)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -30)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -31)--------------MemoryExec: partitions=1, partition_sizes=[1] +10)------CoalesceBatchesExec: target_batch_size=2 +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)--CoalesceBatchesExec: target_batch_size=2 +16)----ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] +18)--------CoalesceBatchesExec: target_batch_size=2 +19)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 +20)------------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] +21)--------------AggregateExec: mode=FinalPartitioned, gby=[id@0 as id, name@1 as name], aggr=[] +22)----------------CoalesceBatchesExec: target_batch_size=2 +23)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +24)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] +25)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +26)------------------------MemoryExec: partitions=1, partition_sizes=[1] +27)--------CoalesceBatchesExec: target_batch_size=2 +28)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 +29)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +30)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index a173db3ea96a..c9207a3034b3 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] 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] -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] -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] -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] -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] -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] +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] +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] +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] +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] +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] @@ -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] @@ -1693,8 +1694,8 @@ logical_plan 06)----------Projection: aggregate_test_100.c3 + aggregate_test_100.c4 AS aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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.c4aggregate_test_100.c4aggregate_test_100.c3@0 as aggregate_test_100.c3 + aggregate_test_100.c4aggregate_test_100.c4aggregate_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 @@ -2137,8 +2138,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 +2184,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] 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] @@ -2551,8 +2552,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] 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] @@ -2713,8 +2714,8 @@ logical_plan 07)------------Projection: CAST(annotated_data_finite.inc_col AS Float64) AS CAST(annotated_data_finite.inc_col AS Float64)annotated_data_finite.inc_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 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] 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] @@ -2767,8 +2768,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] 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] @@ -2814,13 +2815,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 @@ -2861,13 +2861,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 +2960,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 +3362,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] 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 +4065,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] From 5d5cf88ece57b3c45803d807cd773c42d2d2b057 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 29 Apr 2024 16:57:56 +0300 Subject: [PATCH 71/85] Review Part 2 --- datafusion/core/src/physical_planner.rs | 9 +++------ datafusion/physical-expr/src/equivalence/projection.rs | 4 ++-- datafusion/sqllogictest/test_files/select.slt | 1 - datafusion/sqllogictest/test_files/tpch/q3.slt.part | 2 -- datafusion/sqllogictest/test_files/tpch/q8.slt.part | 2 -- datafusion/sqllogictest/test_files/tpch/q9.slt.part | 2 -- datafusion/sqllogictest/test_files/window.slt | 4 ---- 7 files changed, 5 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 25cc9d7c7e12..0a029c381334 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -34,14 +34,11 @@ use crate::datasource::physical_plan::FileSinkConfig; use crate::datasource::source_as_provider; use crate::error::{DataFusionError, Result}; use crate::execution::context::{ExecutionProps, SessionState}; -use crate::logical_expr::expr_vec_fmt; use crate::logical_expr::utils::generate_sort_key; use crate::logical_expr::{ - Aggregate, EmptyRelation, Join, Projection, Sort, TableScan, Unnest, Window, -}; -use crate::logical_expr::{ - Expr, Limit, LogicalPlan, Partitioning as LogicalPartitioning, PlanType, Repartition, - UserDefinedLogicalNode, Values, + expr_vec_fmt, Aggregate, EmptyRelation, Expr, Join, Limit, LogicalPlan, + Partitioning as LogicalPartitioning, PlanType, Projection, Repartition, Sort, + TableScan, Unnest, UserDefinedLogicalNode, Values, Window, }; use crate::physical_expr::{create_physical_expr, create_physical_exprs}; use crate::physical_optimizer::optimizer::PhysicalOptimizerRule; diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index fe80a86f47f9..584a2a06ca56 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -58,7 +58,7 @@ impl ProjectionMapping { .map(|(expr_idx, (expression, name))| { let target_expr = Arc::new(Column::new(&name, expr_idx)) as _; expression - .transform_down(&|e :Arc | match e.as_any().downcast_ref::() { + .transform_down(|e| match e.as_any().downcast_ref::() { Some(col) => { // Sometimes, an expression and its name in the input_schema // doesn't match. This can cause problems, so we make sure @@ -72,7 +72,7 @@ impl ProjectionMapping { } let matching_input_column = Column::new(matching_input_field.name(), idx); - Ok(Transformed::yes(Arc::new(matching_input_column) as _)) + Ok(Transformed::yes(Arc::new(matching_input_column))) } None => Ok(Transformed::no(e)), }) diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index a3a4b3bfc584..c9c04ebdb97d 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1406,7 +1406,6 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true - statement ok drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index e8d352090a24..bb20789f35c0 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -92,8 +92,6 @@ physical_plan 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 - - query IRDI select l_orderkey, diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index c8d67d0d4905..506c68c83f5b 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -163,8 +163,6 @@ physical_plan 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 - - query RR select o_year, diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 82480464370c..687926013ad4 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -128,8 +128,6 @@ physical_plan 50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false - - query TRR select nation, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index c9207a3034b3..3179a86a93df 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1611,7 +1611,6 @@ physical_plan 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true - query III SELECT c9, @@ -1705,7 +1704,6 @@ physical_plan 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true - query III SELECT c3, SUM(c9) OVER(ORDER BY c3+c4 DESC, c9 DESC, c2 ASC) as sum1, @@ -1806,7 +1804,6 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true - query III SELECT c3, SUM(c9) OVER(ORDER BY c3 DESC, c9 DESC, c2 ASC) as sum1, @@ -2093,7 +2090,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, From b826d2991ae063997dbac1ed33d4f4d2c5242949 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 29 Apr 2024 17:33:04 +0300 Subject: [PATCH 72/85] Remove whitespace diffs --- datafusion/sqllogictest/test_files/select.slt | 1 + datafusion/sqllogictest/test_files/tpch/q12.slt.part | 1 + datafusion/sqllogictest/test_files/tpch/q3.slt.part | 2 ++ datafusion/sqllogictest/test_files/tpch/q8.slt.part | 2 ++ datafusion/sqllogictest/test_files/tpch/q9.slt.part | 2 ++ datafusion/sqllogictest/test_files/window.slt | 3 +++ 6 files changed, 11 insertions(+) diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index c9c04ebdb97d..a3a4b3bfc584 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1406,6 +1406,7 @@ physical_plan 04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true + statement ok drop table annotated_data_finite2; diff --git a/datafusion/sqllogictest/test_files/tpch/q12.slt.part b/datafusion/sqllogictest/test_files/tpch/q12.slt.part index a32cac928fe2..49849e8cf065 100644 --- a/datafusion/sqllogictest/test_files/tpch/q12.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q12.slt.part @@ -79,6 +79,7 @@ physical_plan 18)----------------------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_orderpriority], has_header=false + query TII select l_shipmode, diff --git a/datafusion/sqllogictest/test_files/tpch/q3.slt.part b/datafusion/sqllogictest/test_files/tpch/q3.slt.part index bb20789f35c0..e8d352090a24 100644 --- a/datafusion/sqllogictest/test_files/tpch/q3.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q3.slt.part @@ -92,6 +92,8 @@ physical_plan 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 + + query IRDI select l_orderkey, diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index 506c68c83f5b..c8d67d0d4905 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -163,6 +163,8 @@ physical_plan 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 + + query RR select o_year, diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 687926013ad4..82480464370c 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -128,6 +128,8 @@ physical_plan 50)--------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 51)----------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/nation.tbl]]}, projection=[n_nationkey, n_name], has_header=false + + query TRR select nation, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3179a86a93df..a414d8698454 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1611,6 +1611,7 @@ physical_plan 05)--------SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC] 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true + query III SELECT c9, @@ -1704,6 +1705,7 @@ physical_plan 09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true + query III SELECT c3, SUM(c9) OVER(ORDER BY c3+c4 DESC, c9 DESC, c2 ASC) as sum1, @@ -1804,6 +1806,7 @@ physical_plan 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c9], has_header=true + query III SELECT c3, SUM(c9) OVER(ORDER BY c3 DESC, c9 DESC, c2 ASC) as sum1, From fb8bf5ad71394c426ea6b1e75f7924ecc64c370f Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 30 Apr 2024 10:42:40 +0300 Subject: [PATCH 73/85] Do not pushdown projections making calculations --- .../core/src/physical_optimizer/optimize_projections.rs | 7 ++++++- datafusion/sqllogictest/test_files/joins.slt | 8 ++++---- datafusion/sqllogictest/test_files/tpch/q7.slt.part | 4 ++-- datafusion/sqllogictest/test_files/tpch/q8.slt.part | 4 ++-- datafusion/sqllogictest/test_files/tpch/q9.slt.part | 4 ++-- datafusion/sqllogictest/test_files/union.slt | 4 ++-- 6 files changed, 18 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 4c187fe31a99..eda1716b8141 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -460,7 +460,12 @@ impl ProjectionOptimizer { return Ok(Transformed::no(self)); }; // 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() { + if projection.expr().len() >= projection.input().schema().fields().len() + || !projection.expr().iter().all(|(expr, _)| { + expr.as_any().downcast_ref::().is_some() + || expr.as_any().downcast_ref::().is_some() + }) + { return Ok(Transformed::no(self)); } diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index b095b0d49c58..22d6c7f2bf01 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1451,8 +1451,8 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] @@ -1477,8 +1477,8 @@ logical_plan 03)----TableScan: join_t1 projection=[t1_id, t1_name, t1_int] 04)----TableScan: join_t2 projection=[t2_id, t2_name, t2_int] physical_plan -01)CoalesceBatchesExec: target_batch_size=2 -02)--ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +01)ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, t2_id@3 as t2_id, t2_name@4 as t2_name, t2_int@5 as t2_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] +02)--CoalesceBatchesExec: target_batch_size=2 03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 diff --git a/datafusion/sqllogictest/test_files/tpch/q7.slt.part b/datafusion/sqllogictest/test_files/tpch/q7.slt.part index 99d5c934c9ad..f699c3ee6734 100644 --- a/datafusion/sqllogictest/test_files/tpch/q7.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q7.slt.part @@ -91,8 +91,8 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([supp_nation@0, cust_nation@1, l_year@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[supp_nation@0 as supp_nation, cust_nation@1 as cust_nation, l_year@2 as l_year], aggr=[SUM(shipping.volume)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] +08)--------------ProjectionExec: expr=[n_name@3 as supp_nation, n_name@4 as cust_nation, date_part(YEAR, l_shipdate@2) as l_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume] +09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c_nationkey@3, n_nationkey@0)], filter=n_name@0 = FRANCE AND n_name@1 = GERMANY OR n_name@0 = GERMANY AND n_name@1 = FRANCE, projection=[l_extendedprice@0, l_discount@1, l_shipdate@2, n_name@4, n_name@6] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([c_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q8.slt.part b/datafusion/sqllogictest/test_files/tpch/q8.slt.part index c8d67d0d4905..67aa61bef9f7 100644 --- a/datafusion/sqllogictest/test_files/tpch/q8.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q8.slt.part @@ -97,8 +97,8 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([o_year@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[o_year@0 as o_year], aggr=[SUM(CASE WHEN all_nations.nation = Utf8("BRAZIL") THEN all_nations.volume ELSE Int64(0) END), SUM(all_nations.volume)] -08)--------------CoalesceBatchesExec: target_batch_size=8192 -09)----------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] +08)--------------ProjectionExec: expr=[date_part(YEAR, o_orderdate@2) as o_year, l_extendedprice@0 * (Some(1),20,0 - l_discount@1) as volume, n_name@3 as nation] +09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(n_regionkey@3, r_regionkey@0)], projection=[l_extendedprice@0, l_discount@1, o_orderdate@2, n_name@4] 11)--------------------CoalesceBatchesExec: target_batch_size=8192 12)----------------------RepartitionExec: partitioning=Hash([n_regionkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/tpch/q9.slt.part b/datafusion/sqllogictest/test_files/tpch/q9.slt.part index 82480464370c..94ee8ac26b94 100644 --- a/datafusion/sqllogictest/test_files/tpch/q9.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q9.slt.part @@ -84,8 +84,8 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([nation@0, o_year@1], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[nation@0 as nation, o_year@1 as o_year], aggr=[SUM(profit.amount)] -09)----------------CoalesceBatchesExec: target_batch_size=8192 -10)------------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] +09)----------------ProjectionExec: expr=[n_name@5 as nation, date_part(YEAR, o_orderdate@4) as o_year, l_extendedprice@1 * (Some(1),20,0 - l_discount@2) - ps_supplycost@3 * l_quantity@0 as amount] +10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s_nationkey@3, n_nationkey@0)], projection=[l_quantity@0, l_extendedprice@1, l_discount@2, ps_supplycost@4, o_orderdate@5, n_name@7] 12)----------------------CoalesceBatchesExec: target_batch_size=8192 13)------------------------RepartitionExec: partitioning=Hash([s_nationkey@3], 4), input_partitions=4 diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index 6b0d53fb124f..550aebef0bcd 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -290,8 +290,8 @@ physical_plan 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)--CoalesceBatchesExec: target_batch_size=2 -16)----ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +15)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] +16)----CoalesceBatchesExec: target_batch_size=2 17)------HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(CAST(t2.id AS Int32)@2, id@0), (name@1, name@1)], projection=[id@0, name@1] 18)--------CoalesceBatchesExec: target_batch_size=2 19)----------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 From 1611ca1efd8fd0ccd0e19eec797e9a2944a31748 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 30 Apr 2024 13:08:33 +0300 Subject: [PATCH 74/85] Update optimize_projections.rs --- .../core/src/physical_optimizer/optimize_projections.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index eda1716b8141..688d7f57f17f 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -459,7 +459,9 @@ impl ProjectionOptimizer { let Some(projection) = self.plan.as_any().downcast_ref::() else { return Ok(Transformed::no(self)); }; - // If the projection does not narrow the schema, we should not try to push it down: + // 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().downcast_ref::().is_some() From 1ff7d19a0073da271678f3647c8432c339fd601f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 30 Apr 2024 15:38:49 +0300 Subject: [PATCH 75/85] Update module comments for optimize_projections.rs --- .../optimize_projections.rs | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 688d7f57f17f..818c13f3d75f 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -15,40 +15,42 @@ // specific language governing permissions and limitations // under the License. -//! OptimizeProjections rule aims achieving the most effective use of projections +//! 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 root. +//! 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. +//! or when necessary for evaluation or aliasing purposes. //! -//! The optimization is conducted in two phases: +//! The optimization works in two phases: //! //! Top-down Phase: //! --------------- -//! - Traverses the plan from root to leaves. If the node is: -//! 1. Projection node, it may: -//! a. Merge it with its input projection if merge is beneficial. +//! - 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. The projection can be nested into the input. -//! e. Do nothing, otherwise. -//! 2. Non-Projection node: -//! a. Schema needs pruning: Insert the necessary projection into input. -//! b. All fields are required: Do nothing. +//! 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 (resides in with_new_children() implementation of ConcreteTreeNode): +//! Bottom-up Phase: //! ---------------- -//! This pass is required because modifying a plan node can change the column -//! indices used by output nodes. When such a change occurs, we store the old -//! and new indices of the columns in the node's state. We then proceed from -//! the leaves to the root, updating the indices of columns in the plans by -//! referencing these mapping records. After the top-down phase, also some -//! unnecessary projections may emerge. When projections check its input schema -//! mapping, it can remove itself and assign new schema mapping to the new node, -//! which was the projection's input formerly. +//! 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; From 8e2f4879c773fded3a24fbe3a5ea95cc03d923b9 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 30 Apr 2024 17:00:53 +0300 Subject: [PATCH 76/85] Review --- .../optimize_projections.rs | 122 ++++++++---------- 1 file changed, 54 insertions(+), 68 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 818c13f3d75f..594287c1b82b 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -67,7 +67,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; -use datafusion_common::{JoinSide, JoinType}; +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; @@ -220,7 +220,7 @@ impl ProjectionOptimizer { } // Source providers: - // Note: No need to handle source providers separately since if they have projected + // 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(); @@ -276,7 +276,7 @@ impl ProjectionOptimizer { let new_plan = ProjectionExec::try_new(projected_exprs, child_projection.input().clone()) - .map(|e| Arc::new(e) as _)?; + .map(Arc::new)?; Ok(Transformed::yes(Self { plan: new_plan, @@ -407,7 +407,7 @@ impl ProjectionOptimizer { .zip(new_hash_join.schema().fields()) .map(|(index, field)| { ( - Arc::new(Column::new(field.name(), *index)) as Arc, + Arc::new(Column::new(field.name(), *index)) as _, field.name().to_owned(), ) }) @@ -415,7 +415,7 @@ impl ProjectionOptimizer { 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. + // 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 { @@ -429,24 +429,24 @@ impl ProjectionOptimizer { 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: self.children_nodes.swap_remove(0).children_nodes, + children_nodes, })) } else { let new_join_node = Self { plan: new_hash_join, required_columns: IndexSet::new(), schema_mapping: IndexMap::new(), - children_nodes: self.children_nodes.swap_remove(0).children_nodes, + children_nodes, }; - let plan = Arc::new(new_projection) as Arc; Ok(Transformed::no(Self { - plan, + plan: Arc::new(new_projection), required_columns: IndexSet::new(), schema_mapping: IndexMap::new(), children_nodes: vec![new_join_node], @@ -524,7 +524,7 @@ impl ProjectionOptimizer { csv.quote(), csv.escape(), csv.file_compression_type, - )) as _, + )), required_columns: IndexSet::new(), schema_mapping: IndexMap::new(), // Sources cannot have a mapping. children_nodes: vec![], @@ -547,7 +547,7 @@ impl ProjectionOptimizer { csv.quote(), csv.escape(), csv.file_compression_type, - )) as _, + )), required_columns: IndexSet::new(), schema_mapping: IndexMap::new(), // Sources cannot have a mapping. children_nodes: vec![], @@ -652,10 +652,8 @@ impl ProjectionOptimizer { } else { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - let plan = plan.with_new_children(vec![new_child.plan.clone()])?; - self = ProjectionOptimizer { - plan, + plan: plan.with_new_children(vec![new_child.plan.clone()])?, required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], @@ -683,8 +681,7 @@ impl ProjectionOptimizer { // Rewrite the predicate with possibly updated column indices. let new_predicate = update_column_index(filter.predicate(), &schema_mapping); let plan = - Arc::new(FilterExec::try_new(new_predicate, new_child.plan.clone())?) - as _; + Arc::new(FilterExec::try_new(new_predicate, new_child.plan.clone())?); self = ProjectionOptimizer { plan, @@ -724,7 +721,7 @@ impl ProjectionOptimizer { let plan = Arc::new(RepartitionExec::try_new( new_child.plan.clone(), new_partitioning, - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -758,7 +755,7 @@ impl ProjectionOptimizer { SortExec::new(new_sort_exprs, new_child.plan.clone()) .with_preserve_partitioning(sort.preserve_partitioning()) .with_fetch(sort.fetch()), - ) as _; + ); self = ProjectionOptimizer { plan, @@ -796,7 +793,7 @@ impl ProjectionOptimizer { let plan = Arc::new( SortPreservingMergeExec::new(new_sort_exprs, new_child.plan.clone()) .with_fetch(sort_merge.fetch()), - ) as _; + ); self = ProjectionOptimizer { plan, @@ -820,15 +817,15 @@ impl ProjectionOptimizer { let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { let required_columns = mem::take(&mut self.required_columns); - self.children_nodes - .iter_mut() - .for_each(|c| c.required_columns = required_columns.clone()); + for c in self.children_nodes.iter_mut() { + c.required_columns = required_columns.clone(); + } } else { let (new_children, schema_mapping) = self.insert_multi_projection_below_union(requirement_map)?; let plan = Arc::new(UnionExec::new( new_children.iter().map(|c| c.plan.clone()).collect(), - )) as _; + )); self = ProjectionOptimizer { plan, @@ -850,9 +847,9 @@ impl ProjectionOptimizer { let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { let required_columns = mem::take(&mut self.required_columns); - self.children_nodes - .iter_mut() - .for_each(|c| c.required_columns = required_columns.clone()); + for c in self.children_nodes.iter_mut() { + c.required_columns = required_columns.clone(); + } } else { let (new_children, schema_mapping) = self.insert_multi_projection_below_union(requirement_map)?; @@ -904,7 +901,7 @@ impl ProjectionOptimizer { let plan = Arc::new(CrossJoinExec::new( new_left_child.plan.clone(), new_right_child.plan.clone(), - )) as _; + )); self = ProjectionOptimizer { plan, @@ -956,7 +953,7 @@ impl ProjectionOptimizer { let plan = Arc::new(CrossJoinExec::new( left_child.plan.clone(), new_right_child.plan.clone(), - )) as _; + )); self = ProjectionOptimizer { plan, @@ -1398,7 +1395,7 @@ impl ProjectionOptimizer { new_right_child.plan.clone(), new_filter, nlj.join_type(), - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -1426,7 +1423,7 @@ impl ProjectionOptimizer { right_child.plan.clone(), new_filter, nlj.join_type(), - )?) as _; + )?); let new_left_size = new_left_child.plan.schema().fields().len(); left_schema_mapping = extend_left_mapping_with_right( left_schema_mapping, @@ -1464,7 +1461,7 @@ impl ProjectionOptimizer { new_right_child.plan.clone(), new_filter, nlj.join_type(), - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -1499,7 +1496,7 @@ impl ProjectionOptimizer { right_child.plan.clone(), new_filter, nlj.join_type(), - )?) as _; + )?); right_child.required_columns = analyzed_join_right .into_iter() @@ -1543,7 +1540,7 @@ impl ProjectionOptimizer { new_right_child.plan.clone(), new_filter, nlj.join_type(), - )?) as _; + )?); left_child.required_columns = analyzed_join_left .into_iter() @@ -1643,7 +1640,7 @@ impl ProjectionOptimizer { smj.join_type(), smj.sort_options.clone(), smj.null_equals_null, - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -1680,7 +1677,7 @@ impl ProjectionOptimizer { smj.join_type(), smj.sort_options.clone(), smj.null_equals_null, - )?) as _; + )?); let new_left_size = new_left_child.plan.schema().fields().len(); left_schema_mapping = extend_left_mapping_with_right( left_schema_mapping, @@ -1724,7 +1721,7 @@ impl ProjectionOptimizer { smj.join_type(), smj.sort_options.clone(), smj.null_equals_null, - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -1767,7 +1764,7 @@ impl ProjectionOptimizer { smj.join_type(), smj.sort_options.clone(), smj.null_equals_null, - )?) as _; + )?); right_child.required_columns = analyzed_join_right .into_iter() @@ -1819,7 +1816,7 @@ impl ProjectionOptimizer { smj.join_type(), smj.sort_options.clone(), smj.null_equals_null, - )?) as _; + )?); left_child.required_columns = analyzed_join_left .into_iter() @@ -1930,7 +1927,7 @@ impl ProjectionOptimizer { new_left_sort_exprs, new_right_sort_exprs, shj.partition_mode(), - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -1968,7 +1965,7 @@ impl ProjectionOptimizer { shj.left_sort_exprs().map(|exprs| exprs.to_vec()), shj.right_sort_exprs().map(|exprs| exprs.to_vec()), shj.partition_mode(), - )?) as _; + )?); let new_left_size = new_left_child.plan.schema().fields().len(); left_schema_mapping = extend_left_mapping_with_right( left_schema_mapping, @@ -2014,7 +2011,7 @@ impl ProjectionOptimizer { shj.left_sort_exprs().map(|exprs| exprs.to_vec()), shj.right_sort_exprs().map(|exprs| exprs.to_vec()), shj.partition_mode(), - )?) as _; + )?); self = ProjectionOptimizer { plan, @@ -2059,7 +2056,7 @@ impl ProjectionOptimizer { shj.left_sort_exprs().map(|exprs| exprs.to_vec()), shj.right_sort_exprs().map(|exprs| exprs.to_vec()), shj.partition_mode(), - )?) as _; + )?); right_child.required_columns = analyzed_join_right .into_iter() @@ -2113,7 +2110,7 @@ impl ProjectionOptimizer { shj.left_sort_exprs().map(|exprs| exprs.to_vec()), shj.right_sort_exprs().map(|exprs| exprs.to_vec()), shj.partition_mode(), - )?) as _; + )?); left_child.required_columns = analyzed_join_left .into_iter() @@ -2304,10 +2301,9 @@ impl ProjectionOptimizer { }) .collect::>>() else { - return Err(datafusion_common::DataFusionError::Internal( - format!("Window expression {:?} must implement with_new_expressions() API", w_agg - .window_expr()) - )); + return internal_err!( + "Window expression {:?} must implement with_new_expressions() API", w_agg.window_expr() + ); }; let new_keys = w_agg @@ -2315,11 +2311,7 @@ impl ProjectionOptimizer { .iter() .zip(window_usage) .filter_map(|(key, (_column, usage))| { - if usage { - Some(update_column_index(key, &schema_mapping)) - } else { - None - } + usage.then(|| update_column_index(key, &schema_mapping)) }) .collect(); let plan = Arc::new(WindowAggExec::try_new( @@ -2410,10 +2402,9 @@ impl ProjectionOptimizer { }) .collect::>>() else { - return Err(datafusion_common::DataFusionError::Internal( - format!("Bounded window expression {:?} must implement with_new_expressions() API", bw_agg - .window_expr()) - )); + return internal_err!( + "Bounded window expression {:?} must implement with_new_expressions() API", bw_agg.window_expr() + ); }; let new_keys = bw_agg @@ -2421,11 +2412,7 @@ impl ProjectionOptimizer { .iter() .zip(window_usage) .filter_map(|(key, (_column, usage))| { - if usage { - Some(update_column_index(key, &schema_mapping)) - } else { - None - } + usage.then(|| update_column_index(key, &schema_mapping)) }) .collect(); let plan = Arc::new(BoundedWindowAggExec::try_new( @@ -2485,13 +2472,13 @@ impl ProjectionOptimizer { let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); - let inserted_projection = ProjectionOptimizer { + let result = ProjectionOptimizer { plan: inserted_projection, required_columns: new_requirements, schema_mapping: IndexMap::new(), children_nodes: self.children_nodes, }; - Ok((inserted_projection, new_mapping)) + Ok((result, new_mapping)) } /// Multi-child version of `insert_projection` for `UnionExec`'s. @@ -2687,11 +2674,11 @@ impl ProjectionOptimizer { let (used_columns, mut unused_columns) = partition_column_requirements(base); let projected_exprs = convert_projection_exprs(used_columns); - window.iter().for_each(|(col, used)| { + 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(), @@ -2721,14 +2708,13 @@ impl ProjectionOptimizer { let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map .into_iter() .partition(|(column, _used)| column.index() < original_schema_len); - // let mut unused_columns = IndexSet::new(); let (required_cols, mut unused_columns) = partition_column_requirements(base); let projected_exprs = convert_projection_exprs(required_cols); - window.iter().for_each(|(col, used)| { + 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(), From 73967444ee65e6c3d17badab34ca6d634dbc8c23 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 3 May 2024 11:56:55 +0300 Subject: [PATCH 77/85] Unify similar plans --- datafusion-cli/Cargo.lock | 1 + .../optimize_projections.rs | 693 ++++++------------ datafusion/physical-expr-common/Cargo.toml | 1 + .../physical-expr-common/src/aggregate/mod.rs | 10 +- .../src/expressions/column.rs | 36 +- .../physical-expr-common/src/physical_expr.rs | 50 ++ .../physical-expr/src/expressions/mod.rs | 4 +- datafusion/physical-expr/src/lib.rs | 4 +- .../physical-expr/src/window/window_expr.rs | 9 +- datafusion/physical-plan/src/filter.rs | 21 + datafusion/physical-plan/src/lib.rs | 12 + .../physical-plan/src/repartition/mod.rs | 31 + datafusion/physical-plan/src/sorts/sort.rs | 37 +- .../src/sorts/sort_preserving_merge.rs | 36 +- .../src/windows/bounded_window_agg_exec.rs | 38 + .../src/windows/window_agg_exec.rs | 37 + 16 files changed, 523 insertions(+), 497 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 5263b064ff9b..2dd96c3847f6 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1365,6 +1365,7 @@ dependencies = [ "arrow", "datafusion-common", "datafusion-expr", + "indexmap 2.2.6", ] [[package]] diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 594287c1b82b..162e0af29ffb 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -67,13 +67,13 @@ 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_common::{JoinSide, JoinType}; +use datafusion_physical_expr::expressions::{update_expression, Column, Literal}; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::{ - AggregateExpr, LexOrdering, Partitioning, PhysicalExpr, PhysicalExprRef, - PhysicalSortExpr, + AggregateExpr, ExprMapping, ExprWrapper, LexOrdering, Partitioning, PhysicalExpr, + PhysicalExprRef, PhysicalSortExpr, }; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -578,24 +578,15 @@ impl ProjectionOptimizer { } // ------------------------------------------------------------------------ // These plans also preserve the input schema, but may extend requirements. - else if let Some(filter) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_filter(filter)?; - } else if let Some(repartition) = plan.as_any().downcast_ref::() - { - self = self.try_insert_below_repartition(repartition)?; - } else if let Some(sort) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_sort(sort)?; - } else if let Some(sort_merge) = - plan.as_any().downcast_ref::() - { - self = self.try_insert_below_sort_preserving_merge(sort_merge)?; + else if is_plan_requirement_extender(&plan) { + self = self.try_insert_below_req_extender()?; } // ------------------------------------------------------------------------ // Preserves schema and do not change requirements, but have multi-child. - else if plan.as_any().downcast_ref::().is_some() { + else if plan.as_any().downcast_ref::().is_some() + || plan.as_any().downcast_ref::().is_some() + { self = self.try_insert_below_union()?; - } else if plan.as_any().downcast_ref::().is_some() { - self = self.try_insert_below_interleave()?; } // ------------------------------------------------------------------------ // Concatenates schemas and do not change requirements. @@ -619,21 +610,23 @@ impl ProjectionOptimizer { return Ok(self); } self = self.try_insert_below_aggregate(agg)? - } else if let Some(w_agg) = plan.as_any().downcast_ref::() { - if !is_window_expr_rewritable(w_agg.window_expr()) { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan); - return Ok(self); - } - self = self.try_insert_below_window_aggregate(w_agg)? - } else if let Some(bw_agg) = plan.as_any().downcast_ref::() + } else if plan.as_any().downcast_ref::().is_some() + || plan + .as_any() + .downcast_ref::() + .is_some() { - if !is_window_expr_rewritable(bw_agg.window_expr()) { - self.children_nodes[0].required_columns = - collect_columns_in_plan_schema(&self.children_nodes[0].plan); - return Ok(self); + let window_exprs = collect_window_expressions(&self.plan); + self = match self.try_insert_below_window_execs(window_exprs)? { + optimized if optimized.transformed => 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, + ); + return Ok(no_change.data); + } } - self = self.try_insert_below_bounded_window_aggregate(bw_agg)? } else { self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan) @@ -662,141 +655,29 @@ impl ProjectionOptimizer { Ok(self) } - /// Attempts to insert a projection node below a `FilterExec` node. Extends the required columns - /// with those in the filter's predicate and optimizes the plan, potentially inserting a projection node. - fn try_insert_below_filter( - mut self, - filter: &FilterExec, - ) -> Result { - // FilterExec extends the requirements with the columns in its predicate. + fn try_insert_below_req_extender(mut self) -> Result { + let Some(columns) = self.plan.expressions() else { + return Ok(self); + }; self.required_columns - .extend(collect_columns(filter.predicate())); - + .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 (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - // Rewrite the predicate with possibly updated column indices. - let new_predicate = update_column_index(filter.predicate(), &schema_mapping); - let plan = - Arc::new(FilterExec::try_new(new_predicate, new_child.plan.clone())?); - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - - /// Attempts to insert a projection node below a `RepartitionExec` node. If `RepartitionExec` involves - /// a hash repartition, it extends the requirements with the columns in the hashed expressions. - /// The method then optimizes the execution plan accordingly. - fn try_insert_below_repartition( - mut self, - repartition: &RepartitionExec, - ) -> Result { - // If RepartitionExec applies a hash repartition, it extends - // the requirements with the columns in the hashed expressions. - if let Partitioning::Hash(exprs, _size) = repartition.partitioning() { - self.required_columns - .extend(exprs.iter().flat_map(collect_columns)); - } - - 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 (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - // Rewrite the expression if there is any with possibly updated column indices. - let new_partitioning = update_partitioning_expressions( - repartition.partitioning(), - &schema_mapping, - ); - let plan = Arc::new(RepartitionExec::try_new( - new_child.plan.clone(), - new_partitioning, - )?); - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - - /// Attempts to insert a projection node below a `SortExec` node. Extends the requirements with columns - /// involved in the sort expressions and optimizes the execution plan, potentially inserting a projection node. - fn try_insert_below_sort(mut self, sort: &SortExec) -> Result { - // SortExec extends the requirements with the columns in its sort expressions. - self.required_columns.extend( - sort.expr() - .iter() - .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), - ); - - let requirement_map = analyze_requirements(&self); - if all_columns_required(&requirement_map) { - self.children_nodes[0].required_columns = - mem::take(&mut self.required_columns); - } else { - let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - // Rewrite the sort expressions with possibly updated column indices. - let new_sort_exprs = update_sort_expressions(sort.expr(), &schema_mapping); - let plan = Arc::new( - SortExec::new(new_sort_exprs, new_child.plan.clone()) - .with_preserve_partitioning(sort.preserve_partitioning()) - .with_fetch(sort.fetch()), - ); - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - - /// Attempts to insert a projection node below a `SortPreservingMergeExec` node. Similar to `try_insert_below_sort`, - /// it extends the requirements with columns in the sort expressions and optimizes the plan accordingly. - fn try_insert_below_sort_preserving_merge( - mut self, - sort_merge: &SortPreservingMergeExec, - ) -> Result { - // SortPreservingMergeExec extends the requirements with the columns in its sort expressions. - self.required_columns.extend( - sort_merge - .expr() - .iter() - .flat_map(|sort_expr| collect_columns(&sort_expr.expr)), - ); - 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 (new_child, schema_mapping) = self.insert_projection(requirement_map)?; - // Rewrite the sort expressions with possibly updated column indices. - let new_sort_exprs = - update_sort_expressions(sort_merge.expr(), &schema_mapping); - let plan = Arc::new( - SortPreservingMergeExec::new(new_sort_exprs, new_child.plan.clone()) - .with_fetch(sort_merge.fetch()), - ); + let Some(new_plan) = + plan.update_expressions(&expr_mapping(schema_mapping.clone()))? + else { + return Err(datafusion_common::DataFusionError::Internal( + "Plans implementing expressions() must also implement update_expressions()".to_string())); + }; self = ProjectionOptimizer { - plan, + plan: new_plan, required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: vec![new_child], @@ -821,44 +702,15 @@ impl ProjectionOptimizer { c.required_columns = required_columns.clone(); } } else { + let plan = self.plan.clone(); let (new_children, schema_mapping) = self.insert_multi_projection_below_union(requirement_map)?; - let plan = Arc::new(UnionExec::new( + let new_plan = plan.clone().with_new_children( new_children.iter().map(|c| c.plan.clone()).collect(), - )); - - self = ProjectionOptimizer { - plan, - required_columns: IndexSet::new(), // clear the requirements - schema_mapping, - children_nodes: new_children, - } - } - Ok(self) - } - - /// Attempts to insert a projection node below an `InterleaveExec` node in the query execution plan. - /// - /// Similar to `try_insert_below_union`, this method analyzes the requirements of the `InterleaveExec` node and - /// modifies the execution plan accordingly. If all columns are required, it updates the required columns for - /// each child node. Otherwise, it inserts new projection nodes for optimization. This process can lead to a - /// more efficient execution by minimizing the data processed in the context of interleaved execution. - fn try_insert_below_interleave(mut self) -> Result { - 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 = required_columns.clone(); - } - } else { - let (new_children, schema_mapping) = - self.insert_multi_projection_below_union(requirement_map)?; - let plan = Arc::new(InterleaveExec::try_new( - new_children.iter().map(|c| c.plan.clone()).collect(), - )?) as _; + )?; self = ProjectionOptimizer { - plan, + plan: new_plan, required_columns: IndexSet::new(), // clear the requirements schema_mapping, children_nodes: new_children, @@ -2244,186 +2096,75 @@ impl ProjectionOptimizer { Ok(self) } - fn try_insert_below_window_aggregate( + fn try_insert_below_window_execs( mut self, - w_agg: &WindowAggExec, - ) -> Result { - // Both tries to insert a projection to narrow input columns, and tries to narrow the window - // expressions. If none of them survives, we can even remove the window execution plan. - self.required_columns - .extend(w_agg.window_expr().iter().flat_map(|window_expr| { - window_expr - .expressions() - .iter() - .flat_map(collect_columns) - .collect::>() - })); + window_exprs: Vec>, + ) -> Result> { + let Some(columns) = self.plan.expressions() else { + return Ok(Transformed::no(self)); + }; self.required_columns - .extend(w_agg.partition_keys.iter().flat_map(collect_columns)); - let requirement_map = analyze_requirements(&self); + .extend(columns.into_iter().flat_map(|e| collect_columns(&e))); + let requirement_map = analyze_requirements(&self); if !all_columns_required(&requirement_map) { - if window_agg_required( - w_agg.input().schema().fields().len(), + if window_exec_required( + self.plan.children()[0].schema().fields().len(), &requirement_map, ) { - if w_agg - .window_expr() - .iter() - .any(|expr| expr.clone().with_new_expressions(vec![]).is_none()) - { + if window_exprs.iter().any(|expr| { + expr.clone() + .update_expression(&expr_mapping(IndexMap::new())) + .is_none() + }) { self.children_nodes[0].required_columns = self .required_columns .iter() .filter(|col| { col.index() - < w_agg.schema().fields().len() - - w_agg.window_expr().len() + < self.plan.schema().fields().len() - window_exprs.len() }) .cloned() .collect(); - return Ok(self); + return Ok(Transformed::no(self)); } + let plan = self.plan.clone(); let (new_child, schema_mapping, window_usage) = - self.insert_projection_below_window(w_agg, requirement_map)?; - // Rewrite the sort expressions with possibly updated column indices. - let Some(new_window_exprs) = w_agg - .window_expr() - .iter() - .zip(window_usage.clone()) - .filter(|(_window_expr, (_window_col, usage))| *usage) - .map(|(window_expr, (_window_col, _usage))| { - let new_exprs = update_expressions( - &window_expr.expressions(), - &IndexMap::new(), - ); - window_expr.clone().with_new_expressions(new_exprs) - }) - .collect::>>() - else { - return internal_err!( - "Window expression {:?} must implement with_new_expressions() API", w_agg.window_expr() - ); - }; + self.insert_projection_below_window(requirement_map)?; - let new_keys = w_agg - .partition_keys + let mut with_removed_exprs = schema_mapping .iter() - .zip(window_usage) - .filter_map(|(key, (_column, usage))| { - usage.then(|| update_column_index(key, &schema_mapping)) - }) - .collect(); - let plan = Arc::new(WindowAggExec::try_new( - new_window_exprs, - new_child.plan.clone(), - new_keys, - )?) as _; - let required_columns = collect_columns_in_plan_schema(&plan); - self = ProjectionOptimizer { - plan, - required_columns, - schema_mapping, - children_nodes: vec![new_child], - } - } else { - // Remove the WindowAggExec - self = self.children_nodes.swap_remove(0); - self.required_columns = requirement_map - .into_iter() - .filter_map(|(column, used)| if used { Some(column) } else { None }) - .collect(); - } - } else { - self.children_nodes[0].required_columns = - mem::take(&mut self.required_columns) - .into_iter() - .filter(|col| { - col.index() - < w_agg.schema().fields().len() - w_agg.window_expr().len() - }) - .collect(); - } - Ok(self) - } - - fn try_insert_below_bounded_window_aggregate( - mut self, - bw_agg: &BoundedWindowAggExec, - ) -> Result { - // Both tries to insert a projection to narrow input columns, and tries to narrow the window - // expressions. If none of them survives, we can even remove the window execution plan. - self.required_columns - .extend(bw_agg.window_expr().iter().flat_map(|window_expr| { - window_expr - .expressions() - .iter() - .flat_map(collect_columns) - .collect::>() - })); - self.required_columns - .extend(bw_agg.partition_keys.iter().flat_map(collect_columns)); - let requirement_map = analyze_requirements(&self); - if !all_columns_required(&requirement_map) { - if window_agg_required( - bw_agg.input().schema().fields().len(), - &requirement_map, - ) { - if bw_agg - .window_expr() - .iter() - .any(|expr| expr.clone().with_new_expressions(vec![]).is_none()) - { - self.children_nodes[0].required_columns = - mem::take(&mut self.required_columns) - .into_iter() - .filter(|col| { - col.index() - < bw_agg.schema().fields().len() - - bw_agg.window_expr().len() - }) - .collect(); - return Ok(self); - } - let (new_child, schema_mapping, window_usage) = - self.insert_projection_below_bounded_window(bw_agg, requirement_map)?; - // Rewrite the sort expressions with possibly updated column indices. - let Some(new_window_exprs) = bw_agg - .window_expr() - .iter() - .zip(window_usage.clone()) - .filter(|(_window_expr, (_window_col, usage))| *usage) - .map(|(window_expr, (_window_col, _usage))| { - let new_exprs = update_expressions( - &window_expr.expressions(), - &schema_mapping, - ); - window_expr.clone().with_new_expressions(new_exprs) + .map(|(col1, col2)| { + ( + Arc::new(col1.clone()) as Arc, + Some(Arc::new(col2.clone()) as Arc), + ) }) - .collect::>>() + .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(&expr_mapping(schema_mapping.clone()))? else { - return internal_err!( - "Bounded window expression {:?} must implement with_new_expressions() API", bw_agg.window_expr() - ); + return Err(datafusion_common::DataFusionError::Internal( + "Plans implementing expressions() must also implement update_expressions()".to_string())); }; - - let new_keys = bw_agg - .partition_keys - .iter() - .zip(window_usage) - .filter_map(|(key, (_column, usage))| { - usage.then(|| update_column_index(key, &schema_mapping)) - }) - .collect(); - let plan = Arc::new(BoundedWindowAggExec::try_new( - new_window_exprs, - new_child.plan.clone(), - new_keys, - bw_agg.input_order_mode.clone(), - )?) as _; let required_columns = collect_columns_in_plan_schema(&plan); self = ProjectionOptimizer { - plan, + plan: new_plan, required_columns, schema_mapping, children_nodes: vec![new_child], @@ -2442,11 +2183,11 @@ impl ProjectionOptimizer { .into_iter() .filter(|col| { col.index() - < bw_agg.schema().fields().len() - bw_agg.window_expr().len() + < self.plan.schema().fields().len() - window_exprs.len() }) .collect(); } - Ok(self) + Ok(Transformed::no(self)) } /// If a node is known to have redundant columns, we need to insert a projection to its input. @@ -2664,10 +2405,9 @@ impl ProjectionOptimizer { /// `insert_projection` for windows. fn insert_projection_below_window( self, - w_agg: &WindowAggExec, requirement_map: ColumnRequirements, ) -> Result<(Self, IndexMap, ColumnRequirements)> { - let original_schema_len = w_agg.schema().fields().len(); + 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); @@ -2698,42 +2438,6 @@ impl ProjectionOptimizer { Ok((inserted_projection, new_mapping, window)) } - /// `insert_projection` for bounded windows. - fn insert_projection_below_bounded_window( - self, - bw_agg: &BoundedWindowAggExec, - requirement_map: ColumnRequirements, - ) -> Result<(Self, IndexMap, ColumnRequirements)> { - let original_schema_len = bw_agg.schema().fields().len(); - let (base, window): (ColumnRequirements, ColumnRequirements) = requirement_map - .into_iter() - .partition(|(column, _used)| column.index() < original_schema_len); - let (required_cols, mut unused_columns) = partition_column_requirements(base); - let projected_exprs = convert_projection_exprs(required_cols); - 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 @@ -2860,7 +2564,7 @@ impl ProjectionOptimizer { } else if let Some(agg) = plan_any.downcast_ref::() { if agg.aggr_expr().iter().any(|expr| { expr.clone() - .with_new_expressions(expr.expressions()) + .update_expression(&expr_mapping(all_mappings[0].clone())) .is_none() && !self.children_nodes[0].schema_mapping.is_empty() }) { @@ -2945,9 +2649,9 @@ impl ProjectionOptimizer { .iter() .map(|child| child.plan.clone()) .collect(), - ); + )?; - self.plan = res?; + self.plan = res; } Ok(Transformed::yes(self)) @@ -3718,6 +3422,20 @@ fn collect_right_hj_right_requirements( 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, @@ -3803,45 +3521,25 @@ fn update_expr_with_projection( new_expr.map(|e| (state == RewriteState::RewrittenValid).then_some(e.data)) } -/// Rewrites the expressions with new index values. -fn update_expressions( - exprs: &[Arc], - mapping: &IndexMap, -) -> Vec> { - exprs - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect::>() -} - /// Rewrites the sort expressions with new index values. fn update_sort_expressions( sort_exprs: &[PhysicalSortExpr], mapping: &IndexMap, ) -> LexOrdering { + let expr_map = expr_mapping(mapping.clone()); sort_exprs .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, mapping), - options: sort_expr.options, + .filter_map(|sort_expr| { + update_expression(sort_expr.expr.clone(), &expr_map).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, + } + }) }) .collect::>() } -/// Updates the expressions subject to hashing of the `Partitioning` according to -/// the mapping. If it is not a hash partitioning, they remains as they are. -fn update_partitioning_expressions( - partitioning: &Partitioning, - mapping: &IndexMap, -) -> Partitioning { - if let Partitioning::Hash(exprs, size) = partitioning { - let updated_exprs = update_expressions(exprs, mapping); - Partitioning::Hash(updated_exprs, *size) - } else { - partitioning.clone() - } -} - /// Rewrites the window expressions with new index values. fn update_window_exprs( window_exprs: &[Arc], @@ -3850,8 +3548,9 @@ fn update_window_exprs( window_exprs .iter() .map(|window_expr| { - let new_exprs = update_expressions(&window_expr.expressions(), mapping); - window_expr.clone().with_new_expressions(new_exprs) + window_expr + .clone() + .update_expression(&expr_mapping(mapping.clone())) }) .collect::>>() } @@ -3864,13 +3563,9 @@ fn update_aggregate_exprs( aggregate_exprs .iter() .map(|aggr_expr| { - aggr_expr.clone().with_new_expressions( - aggr_expr - .expressions() - .iter() - .map(|expr| update_column_index(expr, mapping)) - .collect(), - ) + aggr_expr + .clone() + .update_expression(&expr_mapping(mapping.clone())) }) .collect::>>() } @@ -3881,44 +3576,20 @@ fn update_join_on( left_mapping: &IndexMap, right_mapping: &IndexMap, ) -> JoinOn { + let left_expr_map = expr_mapping(left_mapping.clone()); + let right_expr_map = expr_mapping(right_mapping.clone()); join_on .iter() - .map(|(left, right)| { - ( - update_column_index(left, left_mapping), - update_column_index(right, right_mapping), - ) - }) - .collect() -} - -/// Given mapping representing the initial and new index values, -/// it updates the indices of columns in the [`PhysicalExpr`]. -fn update_column_index( - expr: &Arc, - mapping: &IndexMap, -) -> Arc { - 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)); - }; - state = RewriteState::RewrittenValid; - // Update the index of `column`: - if let Some(updated) = mapping.get(column) { - Ok(Transformed::yes(Arc::new(updated.clone()) as _)) - } else { - Ok(Transformed::no(expr.clone())) + .filter_map(|(left, right)| { + match ( + update_expression(left.clone(), &left_expr_map), + update_expression(right.clone(), &right_expr_map), + ) { + (Some(left), Some(right)) => Some((left, right)), + _ => None, } }) - .unwrap() - .data; - new_expr + .collect() } /// Updates the equivalence conditions of the joins according to the new indices of columns. @@ -4353,8 +4024,14 @@ fn rewrite_filter( input_plan: Arc, mapping: &IndexMap, ) -> Result> { - FilterExec::try_new(update_column_index(predicate, mapping), input_plan) - .map(|plan| Arc::new(plan) as _) + let Some(new_expr) = + update_expression(predicate.clone(), &expr_mapping(mapping.clone())) + else { + return Err(datafusion_common::DataFusionError::Internal( + "Filter predicate cannot be rewritten".to_string(), + )); + }; + FilterExec::try_new(new_expr, input_plan).map(|plan| Arc::new(plan) as _) } fn rewrite_hj_filter( @@ -4405,7 +4082,10 @@ fn rewrite_projection( projection .expr() .iter() - .map(|(expr, alias)| (update_column_index(expr, mapping), alias.clone())) + .filter_map(|(expr, alias)| { + update_expression(expr.clone(), &expr_mapping(mapping.clone())) + .map(|e| (e, alias.clone())) + }) .collect::>(), input_plan, ) @@ -4419,7 +4099,10 @@ fn rewrite_repartition( mapping: &IndexMap, ) -> Result> { let new_partitioning = if let Partitioning::Hash(exprs, size) = partitioning { - let new_exprs = update_expressions(exprs, mapping); + let new_exprs = exprs + .iter() + .filter_map(|e| update_expression(e.clone(), &expr_mapping(mapping.clone()))) + .collect(); Partitioning::Hash(new_exprs, *size) } else { partitioning.clone() @@ -4601,21 +4284,31 @@ fn rewrite_symmetric_hash_join( ) }); + let left_expr_map = expr_mapping(left_mapping.clone()); + let right_expr_map = expr_mapping(right_mapping.clone()); let new_left_sort_exprs = shj.left_sort_exprs().map(|exprs| { exprs .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, left_mapping), - options: sort_expr.options, + .filter_map(|sort_expr| { + update_expression(sort_expr.expr.clone(), &left_expr_map).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, + } + }) }) .collect() }); let new_right_sort_exprs = shj.left_sort_exprs().map(|exprs| { exprs .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: update_column_index(&sort_expr.expr, right_mapping), - options: sort_expr.options, + .filter_map(|sort_expr| { + update_expression(sort_expr.expr.clone(), &right_expr_map).map(|expr| { + PhysicalSortExpr { + expr, + options: sort_expr.options, + } + }) }) .collect() }); @@ -4640,16 +4333,21 @@ fn rewrite_aggregate( input_plan: Arc, mapping: &IndexMap, ) -> Result>> { + let expr_map = expr_mapping(mapping.clone()); let new_group_by = PhysicalGroupBy::new( agg.group_expr() .expr() .iter() - .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .filter_map(|(expr, alias)| { + update_expression(expr.clone(), &expr_map).map(|e| (e, alias.to_string())) + }) .collect(), agg.group_expr() .null_expr() .iter() - .map(|(expr, alias)| (update_column_index(expr, mapping), alias.to_string())) + .filter_map(|(expr, alias)| { + update_expression(expr.clone(), &expr_map).map(|e| (e, alias.to_string())) + }) .collect(), agg.group_expr().groups().to_vec(), ); @@ -4662,10 +4360,10 @@ fn rewrite_aggregate( let new_filter = agg .filter_expr() .iter() - .map(|opt_expr| { + .filter_map(|opt_expr| { opt_expr .clone() - .map(|expr| update_column_index(&expr, mapping)) + .map(|expr| update_expression(expr, &expr_mapping(mapping.clone()))) }) .collect(); AggregateExec::try_new( @@ -4692,7 +4390,11 @@ fn rewrite_window_aggregate( } else { return Ok(None); }; - let new_partition_keys = update_expressions(&w_agg.partition_keys, mapping); + let new_partition_keys = w_agg + .partition_keys + .iter() + .filter_map(|k| update_expression(k.clone(), &expr_mapping(mapping.clone()))) + .collect(); WindowAggExec::try_new(new_window, input_plan, new_partition_keys) .map(|plan| Some(Arc::new(plan) as _)) } @@ -4710,7 +4412,11 @@ fn rewrite_bounded_window_aggregate( } else { return Ok(None); }; - let new_partition_keys = update_expressions(&bw_agg.partition_keys, mapping); + let new_partition_keys = bw_agg + .partition_keys + .iter() + .filter_map(|k| update_expression(k.clone(), &expr_mapping(mapping.clone()))) + .collect(); BoundedWindowAggExec::try_new( new_window, input_plan, @@ -4731,6 +4437,14 @@ fn is_plan_schema_agnostic(plan: &Arc) -> bool { || plan_any.downcast_ref::().is_some() } +fn is_plan_requirement_extender(plan: &Arc) -> bool { + let plan_any = plan.as_any(); + plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() + || plan_any.downcast_ref::().is_some() +} + /// 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 { @@ -4757,11 +4471,6 @@ fn is_agg_expr_rewritable(aggr_expr: &[Arc<(dyn AggregateExpr)>]) -> bool { aggr_expr.iter().all(|expr| expr.expressions().is_empty()) } -/// Tries to rewrite the [`WindowExpr`] with the existing expressions to keep on optimization. -fn is_window_expr_rewritable(window_expr: &[Arc<(dyn WindowExpr)>]) -> bool { - window_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 { @@ -5000,7 +4709,7 @@ fn preserve_requirements(po: ProjectionOptimizer) -> Result } } -fn window_agg_required( +fn window_exec_required( original_schema_len: usize, requirements: &ColumnRequirements, ) -> bool { @@ -5138,6 +4847,24 @@ fn index_changes_after_projection_removal( .collect() } +fn expr_mapping(schema_mapping: IndexMap) -> ExprMapping { + ExprMapping { + map: schema_mapping + .iter() + .map(|(e1, e2)| { + ( + ExprWrapper { + expr: Arc::new(e1.clone()), + }, + Some(ExprWrapper { + expr: Arc::new(e2.clone()), + }), + ) + }) + .collect(), + } +} + #[cfg(test)] mod tests { use super::*; @@ -5612,7 +5339,7 @@ mod tests { let expected = [ "CoalescePartitionsExec", - " ProjectionExec: expr=[b@1 as b, a@0 as a_new, d@2 as d]", + " 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); 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 492531e7a7e1..4c285281e52f 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -26,7 +26,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}; @@ -145,11 +145,11 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { not_impl_err!("Retractable Accumulator hasn't been implemented for {self:?} yet") } - /// Rewrites the aggregate expression with the given expressions. - /// The order of the given expressions is taken into account while replacing. - fn with_new_expressions( + /// 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, - _expressions: Vec>, + _map: &ExprMapping, ) -> Option> { None } diff --git a/datafusion/physical-expr-common/src/expressions/column.rs b/datafusion/physical-expr-common/src/expressions/column.rs index 2cd52d6332fb..3c1b45d03728 100644 --- a/datafusion/physical-expr-common/src/expressions/column.rs +++ b/datafusion/physical-expr-common/src/expressions/column.rs @@ -25,10 +25,11 @@ use arrow::{ datatypes::{DataType, Schema}, record_batch::RecordBatch, }; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; -use crate::physical_expr::{down_cast_any_ref, PhysicalExpr}; +use crate::physical_expr::{down_cast_any_ref, ExprMapping, PhysicalExpr}; /// Represents the column at a given index in a RecordBatch #[derive(Debug, Hash, PartialEq, Eq, Clone)] @@ -135,3 +136,36 @@ impl Column { pub fn col(name: &str, schema: &Schema) -> Result> { Ok(Arc::new(Column::new_with_schema(name, schema)?)) } + +pub fn update_expression( + expr: Arc, + map: &ExprMapping, +) -> Option> { + if let Some(new_expr) = map.get(expr.clone()) { + new_expr + } else { + update_columns(expr, map) + } +} + +fn update_columns( + expr: Arc, + map: &ExprMapping, +) -> Option> { + if expr.as_any().downcast_ref::().is_some() { + map.get(expr.clone()).unwrap_or(Some(expr)) + } else { + Some( + expr.transform_up(|e: Arc| { + if e.as_any().downcast_ref::().is_some() { + if let Some(Some(updated)) = map.get(e.clone()) { + return Ok(Transformed::yes(updated)); + } + } + Ok(Transformed::no(e)) + }) + .unwrap() + .data, + ) + } +} diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index be6358e73c99..1dbb4356d6ea 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -28,6 +28,7 @@ use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::ColumnarValue; +use indexmap::IndexMap; use crate::sort_properties::SortProperties; use crate::utils::scatter; @@ -209,3 +210,52 @@ pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { any } } + +#[derive(Debug)] +pub struct ExprMapping { + pub map: IndexMap>, +} + +impl ExprMapping { + pub fn insert( + &mut self, + key: Arc, + value: Option>, + ) -> Option>> { + self.map + .insert( + ExprWrapper { expr: key }, + value.map(|v| ExprWrapper { expr: v }), + ) + .map(|wrapper| wrapper.map(|w| w.expr)) + } + + pub fn get( + &self, + key: Arc, + ) -> Option>> { + let key_wrapper = ExprWrapper { expr: key }; + self.map + .get(&key_wrapper) + .map(|value| value.as_ref().map(|v| v.expr.clone())) + } +} + +#[derive(Debug)] +pub struct ExprWrapper { + pub expr: Arc, +} + +impl PartialEq for ExprWrapper { + fn eq(&self, other: &ExprWrapper) -> bool { + self.expr.eq(&other.expr) + } +} + +impl Eq for ExprWrapper {} + +impl Hash for ExprWrapper { + fn hash(&self, state: &mut H) { + self.expr.hash(state); + } +} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 55ebd9ed8c44..488fdee52732 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -84,7 +84,9 @@ pub use case::{case, CaseExpr}; pub use cast::{cast, cast_with_options, CastExpr}; pub use column::UnKnownColumn; pub use datafusion_expr::utils::format_state_name; -pub use datafusion_physical_expr_common::expressions::column::{col, Column}; +pub use datafusion_physical_expr_common::expressions::column::{ + col, update_expression, Column, +}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e0f19ad133e5..e5647d06dd74 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -47,7 +47,9 @@ pub use physical_expr::{ PhysicalExprRef, }; -pub use datafusion_physical_expr_common::physical_expr::PhysicalExpr; +pub use datafusion_physical_expr_common::physical_expr::{ + ExprMapping, ExprWrapper, 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 fd52beb60fe5..e77041789b1b 100644 --- a/datafusion/physical-expr/src/window/window_expr.rs +++ b/datafusion/physical-expr/src/window/window_expr.rs @@ -34,6 +34,7 @@ use datafusion_expr::window_state::{ }; use datafusion_expr::{Accumulator, PartitionEvaluator, WindowFrame, WindowFrameBound}; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use indexmap::IndexMap; /// Common trait for [window function] implementations @@ -129,11 +130,11 @@ pub trait WindowExpr: Send + Sync + Debug { /// Get the reverse expression of this [WindowExpr]. fn get_reverse_expr(&self) -> Option>; - /// Rewrites the window expression with the given expressions. - /// The order of the given expressions is taken into account while replacing. - fn with_new_expressions( + /// 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, - _expressions: Vec>, + _map: &ExprMapping, ) -> Option> { None } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index bf1ab8b73126..b0fb6d011aff 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -47,6 +47,8 @@ use datafusion_physical_expr::{ analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, }; +use datafusion_physical_expr_common::expressions::column::update_expression; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -287,6 +289,25 @@ 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) = update_expression(self.predicate.clone(), map) else { + return Err(DataFusionError::Internal( + "Filter predicate cannot be empty".to_string(), + )); + }; + Ok(Some(Arc::new(FilterExec::try_new( + new_predicate, + self.input.clone(), + )?))) + } } /// This function ensures that all bounds in the `ExprBoundaries` vector are diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index cd2be33e86c1..9aef0c303c25 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -37,6 +37,7 @@ use datafusion_physical_expr::{ EquivalenceProperties, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::TryStreamExt; use tokio::task::JoinSet; @@ -427,6 +428,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/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index b6554f46cf78..79a77894fe4f 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -48,6 +48,8 @@ use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_expr_common::expressions::column::update_expression; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use hashbrown::HashMap; @@ -660,6 +662,35 @@ 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| update_expression(expr.clone(), map)) + .collect::>(); + Partitioning::Hash(updated_exprs.into_iter().flatten().collect(), *size) + } else { + self.partitioning.clone() + }; + + Ok(Some(Arc::new(RepartitionExec::try_new( + self.input.clone(), + new_partitioning, + )?))) + } } impl RepartitionExec { diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index ebeaf9e471c3..62831b67c7b0 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -56,8 +56,10 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; +use datafusion_physical_expr_common::expressions::column::update_expression; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; use tokio::sync::mpsc::Sender; @@ -991,6 +993,39 @@ 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| { + update_expression(sort_expr.expr.clone(), map).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..dcd814858a17 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -32,8 +32,10 @@ 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::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr_common::expressions::column::update_expression; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -257,6 +259,38 @@ 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| { + update_expression(sort_expr.expr.clone(), map).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 b1c306194813..5d8e5b0f63dc 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -61,6 +61,8 @@ use datafusion_physical_expr::window::{ use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; use ahash::RandomState; +use datafusion_physical_expr_common::expressions::column::update_expression; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; @@ -333,6 +335,42 @@ 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| update_expression(key.clone(), map)) + .collect(); + + Ok(Some(Arc::new(BoundedWindowAggExec::try_new( + new_window_exprs, + self.input.clone(), + new_keys, + self.input_order_mode.clone(), + )?))) + } } /// Trait that specifies how we search for (or calculate) partitions. It has two diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 46ba21bd797e..1550549c964b 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -46,6 +46,8 @@ use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr_common::expressions::column::update_expression; +use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::{ready, Stream, StreamExt}; /// Window execution plan @@ -264,6 +266,41 @@ 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| update_expression(key.clone(), map)) + .collect(); + + Ok(Some(Arc::new(WindowAggExec::try_new( + new_window_exprs, + self.input.clone(), + new_keys, + )?))) + } } fn create_schema( From 8bdd12f91675d142f9cba2cfb2f304510ef45826 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 3 May 2024 12:23:25 +0300 Subject: [PATCH 78/85] Update optimize_projections.rs --- .../optimize_projections.rs | 90 +++++++++++++------ datafusion/sqllogictest/test_files/window.slt | 20 ++--- 2 files changed, 74 insertions(+), 36 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 162e0af29ffb..16986d1a3e7d 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -699,7 +699,7 @@ impl ProjectionOptimizer { 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 = required_columns.clone(); + c.required_columns.clone_from(&required_columns); } } else { let plan = self.plan.clone(); @@ -3058,7 +3058,7 @@ impl PhysicalOptimizerRule for OptimizeProjections { let mut optimizer = ProjectionOptimizer::new_default(final_schema_determinant); // Insert the initial requirements to the root node, and run the rule. - optimizer.required_columns = initial_requirements.clone(); + optimizer.required_columns.clone_from(&initial_requirements); let mut optimized = optimizer.transform_down(|o: ProjectionOptimizer| { o.adjust_node_with_requirements().map(Transformed::yes) })?; @@ -3906,7 +3906,9 @@ fn update_hj_left_child( }; let mut right_node = children.swap_remove(0); - right_node.required_columns = hj_right_requirements.clone(); + right_node + .required_columns + .clone_from(hj_right_requirements); Ok((new_left_node, right_node)) } @@ -3941,7 +3943,7 @@ fn update_hj_right_child( }; let mut left_node = children.swap_remove(0); - left_node.required_columns = hj_left_requirements.clone(); + left_node.required_columns.clone_from(hj_left_requirements); Ok((left_node, new_right_node)) } @@ -4868,6 +4870,7 @@ fn expr_mapping(schema_mapping: IndexMap) -> ExprMapping { #[cfg(test)] mod tests { use super::*; + use std::any::Any; use std::sync::Arc; use crate::datasource::file_format::file_compression_type::FileCompressionType; @@ -4886,9 +4889,14 @@ mod tests { use arrow_schema::{DataType, Field, Schema, SortOptions}; use datafusion_common::config::ConfigOptions; - use datafusion_common::{JoinSide, JoinType, Result, ScalarValue, Statistics}; + use datafusion_common::{ + plan_err, JoinSide, JoinType, Result, ScalarValue, Statistics, + }; use datafusion_execution::object_store::ObjectStoreUrl; - use datafusion_expr::{Operator, ScalarFunctionDefinition, WindowFrame}; + use datafusion_expr::{ + ColumnarValue, Operator, ScalarFunctionDefinition, ScalarUDF, ScalarUDFImpl, + Signature, Volatility, WindowFrame, + }; use datafusion_physical_expr::expressions::{ rank, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, RowNumber, Sum, @@ -4903,6 +4911,44 @@ mod tests { 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), @@ -4975,9 +5021,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::Name( - "dummy".to_owned().into_boxed_str().into(), - ), + ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5043,9 +5087,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::Name( - "dummy".to_owned().into_boxed_str().into(), - ), + ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5114,9 +5156,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::Name( - "dummy".to_owned().into_boxed_str().into(), - ), + ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5182,9 +5222,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::Name( - "dummy".to_owned().into_boxed_str().into(), - ), + ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b_new", 1)), @@ -5782,7 +5820,7 @@ mod tests { 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]", + " 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); @@ -5792,7 +5830,7 @@ mod tests { 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]", + " 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); @@ -5962,7 +6000,7 @@ mod tests { "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]", + " 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"]; @@ -5975,7 +6013,7 @@ mod tests { let expected = [ "FilterExec: sum@0 > 0", " ProjectionExec: expr=[c@0 + x@1 as sum]", - " SortExec: expr=[c@0 ASC,x@1 ASC]", + " 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"]; @@ -6091,7 +6129,7 @@ mod tests { 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]", + " 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", @@ -6107,7 +6145,7 @@ mod tests { "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]", + " 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", @@ -6228,7 +6266,7 @@ mod tests { 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]", + " 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", @@ -6244,7 +6282,7 @@ mod tests { "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]", + " 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", diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 55dab365faa7..cdaa86f4316b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1523,17 +1523,17 @@ physical_plan 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] +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] +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] +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] +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] +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 @@ -2185,7 +2185,7 @@ logical_plan physical_plan 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] +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] 06)----------ProjectionExec: expr=[c1@0 as c1, c9@2 as c9, c12@3 as c12, 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@4 as 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] @@ -2553,7 +2553,7 @@ logical_plan physical_plan 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] +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] 06)----------ProjectionExec: expr=[CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col@0 as CAST(annotated_data_finite.desc_col AS Int64)annotated_data_finite.desc_col, inc_col@3 as inc_col, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@5 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@6 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@7 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@8 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@9 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@10 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@12 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@13 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@14 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@15 as COUNT(*) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@16 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@17 as SUM(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@20 as MIN(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@21 as MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@22 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@23 as MAX(annotated_data_finite.desc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@25 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING, COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as COUNT(*) ORDER BY [annotated_data_finite.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING] @@ -2715,7 +2715,7 @@ logical_plan physical_plan 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] +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] 06)----------BoundedWindowAggExec: wdw=[SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MIN(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "MAX(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "COUNT(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "AVG(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] @@ -2769,7 +2769,7 @@ logical_plan physical_plan 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] +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] 06)----------BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "FIRST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }, LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "LAST_VALUE(annotated_data_finite.inc_col) ORDER BY [annotated_data_finite.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED 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(3)), is_causal: false }], mode=[Sorted] @@ -3363,7 +3363,7 @@ logical_plan physical_plan 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] +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] 06)----------SortExec: expr=[c12@1 ASC NULLS LAST], preserve_partitioning=[false] From 8fd557091b01b31a7c4a5e7562251d1b19162d57 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 6 May 2024 13:30:44 +0300 Subject: [PATCH 79/85] Refactor ExprMapping --- .../optimize_projections.rs | 267 +++++++++--------- .../src/expressions/column.rs | 38 +-- .../physical-expr-common/src/physical_expr.rs | 120 +++++--- .../src/equivalence/properties.rs | 20 +- .../physical-expr/src/expressions/mod.rs | 4 +- datafusion/physical-expr/src/lib.rs | 4 +- .../physical-expr/src/window/window_expr.rs | 3 +- datafusion/physical-plan/src/filter.rs | 24 +- datafusion/physical-plan/src/lib.rs | 4 +- .../physical-plan/src/repartition/mod.rs | 14 +- datafusion/physical-plan/src/sorts/sort.rs | 20 +- .../src/sorts/sort_preserving_merge.rs | 11 +- .../src/windows/bounded_window_agg_exec.rs | 18 +- .../src/windows/window_agg_exec.rs | 13 +- 14 files changed, 257 insertions(+), 303 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 16986d1a3e7d..2e99e1feaf5b 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -68,12 +68,12 @@ use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; use datafusion_common::{JoinSide, JoinType}; -use datafusion_physical_expr::expressions::{update_expression, Column, Literal}; +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, ExprWrapper, LexOrdering, Partitioning, PhysicalExpr, - PhysicalExprRef, PhysicalSortExpr, + AggregateExpr, ExprMapping, LexOrdering, Partitioning, PhysicalExpr, PhysicalExprRef, + PhysicalSortExpr, }; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, @@ -670,7 +670,7 @@ impl ProjectionOptimizer { let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; let Some(new_plan) = - plan.update_expressions(&expr_mapping(schema_mapping.clone()))? + plan.update_expressions(&ExprMapping::new(schema_mapping.clone()))? else { return Err(datafusion_common::DataFusionError::Internal( "Plans implementing expressions() must also implement update_expressions()".to_string())); @@ -2114,7 +2114,7 @@ impl ProjectionOptimizer { ) { if window_exprs.iter().any(|expr| { expr.clone() - .update_expression(&expr_mapping(IndexMap::new())) + .update_expression(&ExprMapping::new(IndexMap::new())) .is_none() }) { self.children_nodes[0].required_columns = self @@ -2157,7 +2157,7 @@ impl ProjectionOptimizer { }); let Some(new_plan) = plan .clone() - .update_expressions(&expr_mapping(schema_mapping.clone()))? + .update_expressions(&ExprMapping::new(schema_mapping.clone()))? else { return Err(datafusion_common::DataFusionError::Internal( "Plans implementing expressions() must also implement update_expressions()".to_string())); @@ -2564,7 +2564,7 @@ impl ProjectionOptimizer { } else if let Some(agg) = plan_any.downcast_ref::() { if agg.aggr_expr().iter().any(|expr| { expr.clone() - .update_expression(&expr_mapping(all_mappings[0].clone())) + .update_expression(&ExprMapping::new(all_mappings[0].clone())) .is_none() && !self.children_nodes[0].schema_mapping.is_empty() }) { @@ -3526,16 +3526,16 @@ fn update_sort_expressions( sort_exprs: &[PhysicalSortExpr], mapping: &IndexMap, ) -> LexOrdering { - let expr_map = expr_mapping(mapping.clone()); + let expr_map = ExprMapping::new(mapping.clone()); sort_exprs .iter() .filter_map(|sort_expr| { - update_expression(sort_expr.expr.clone(), &expr_map).map(|expr| { - PhysicalSortExpr { + expr_map + .update_expression(sort_expr.expr.clone()) + .map(|expr| PhysicalSortExpr { expr, options: sort_expr.options, - } - }) + }) }) .collect::>() } @@ -3550,7 +3550,7 @@ fn update_window_exprs( .map(|window_expr| { window_expr .clone() - .update_expression(&expr_mapping(mapping.clone())) + .update_expression(&ExprMapping::new(mapping.clone())) }) .collect::>>() } @@ -3565,7 +3565,7 @@ fn update_aggregate_exprs( .map(|aggr_expr| { aggr_expr .clone() - .update_expression(&expr_mapping(mapping.clone())) + .update_expression(&ExprMapping::new(mapping.clone())) }) .collect::>>() } @@ -3576,14 +3576,14 @@ fn update_join_on( left_mapping: &IndexMap, right_mapping: &IndexMap, ) -> JoinOn { - let left_expr_map = expr_mapping(left_mapping.clone()); - let right_expr_map = expr_mapping(right_mapping.clone()); + 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 ( - update_expression(left.clone(), &left_expr_map), - update_expression(right.clone(), &right_expr_map), + left_expr_map.update_expression(left.clone()), + right_expr_map.update_expression(right.clone()), ) { (Some(left), Some(right)) => Some((left, right)), _ => None, @@ -4026,9 +4026,8 @@ fn rewrite_filter( input_plan: Arc, mapping: &IndexMap, ) -> Result> { - let Some(new_expr) = - update_expression(predicate.clone(), &expr_mapping(mapping.clone())) - else { + let map = ExprMapping::new(mapping.clone()); + let Some(new_expr) = map.update_expression(predicate.clone()) else { return Err(datafusion_common::DataFusionError::Internal( "Filter predicate cannot be rewritten".to_string(), )); @@ -4080,12 +4079,14 @@ fn rewrite_projection( input_plan: Arc, mapping: &IndexMap, ) -> Result> { + let mapping = ExprMapping::new(mapping.clone()); ProjectionExec::try_new( projection .expr() .iter() .filter_map(|(expr, alias)| { - update_expression(expr.clone(), &expr_mapping(mapping.clone())) + mapping + .update_expression(expr.clone()) .map(|e| (e, alias.clone())) }) .collect::>(), @@ -4101,9 +4102,10 @@ fn rewrite_repartition( 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| update_expression(e.clone(), &expr_mapping(mapping.clone()))) + .filter_map(|e| mapping.update_expression(e.clone())) .collect(); Partitioning::Hash(new_exprs, *size) } else { @@ -4286,18 +4288,18 @@ fn rewrite_symmetric_hash_join( ) }); - let left_expr_map = expr_mapping(left_mapping.clone()); - let right_expr_map = expr_mapping(right_mapping.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| { - update_expression(sort_expr.expr.clone(), &left_expr_map).map(|expr| { - PhysicalSortExpr { + left_expr_map + .update_expression(sort_expr.expr.clone()) + .map(|expr| PhysicalSortExpr { expr, options: sort_expr.options, - } - }) + }) }) .collect() }); @@ -4305,12 +4307,12 @@ fn rewrite_symmetric_hash_join( exprs .iter() .filter_map(|sort_expr| { - update_expression(sort_expr.expr.clone(), &right_expr_map).map(|expr| { - PhysicalSortExpr { + right_expr_map + .update_expression(sort_expr.expr.clone()) + .map(|expr| PhysicalSortExpr { expr, options: sort_expr.options, - } - }) + }) }) .collect() }); @@ -4335,20 +4337,24 @@ fn rewrite_aggregate( input_plan: Arc, mapping: &IndexMap, ) -> Result>> { - let expr_map = expr_mapping(mapping.clone()); + let expr_map = ExprMapping::new(mapping.clone()); let new_group_by = PhysicalGroupBy::new( agg.group_expr() .expr() .iter() .filter_map(|(expr, alias)| { - update_expression(expr.clone(), &expr_map).map(|e| (e, alias.to_string())) + expr_map + .update_expression(expr.clone()) + .map(|e| (e, alias.to_string())) }) .collect(), agg.group_expr() .null_expr() .iter() .filter_map(|(expr, alias)| { - update_expression(expr.clone(), &expr_map).map(|e| (e, alias.to_string())) + expr_map + .update_expression(expr.clone()) + .map(|e| (e, alias.to_string())) }) .collect(), agg.group_expr().groups().to_vec(), @@ -4359,13 +4365,12 @@ fn rewrite_aggregate( } 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| update_expression(expr, &expr_mapping(mapping.clone()))) + opt_expr.clone().map(|expr| mapping.update_expression(expr)) }) .collect(); AggregateExec::try_new( @@ -4392,10 +4397,11 @@ fn rewrite_window_aggregate( } else { return Ok(None); }; + let mapping = ExprMapping::new(mapping.clone()); let new_partition_keys = w_agg .partition_keys .iter() - .filter_map(|k| update_expression(k.clone(), &expr_mapping(mapping.clone()))) + .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 _)) @@ -4414,10 +4420,11 @@ fn rewrite_bounded_window_aggregate( } else { return Ok(None); }; + let mapping = ExprMapping::new(mapping.clone()); let new_partition_keys = bw_agg .partition_keys .iter() - .filter_map(|k| update_expression(k.clone(), &expr_mapping(mapping.clone()))) + .filter_map(|k| mapping.update_expression(k.clone())) .collect(); BoundedWindowAggExec::try_new( new_window, @@ -4849,24 +4856,6 @@ fn index_changes_after_projection_removal( .collect() } -fn expr_mapping(schema_mapping: IndexMap) -> ExprMapping { - ExprMapping { - map: schema_mapping - .iter() - .map(|(e1, e2)| { - ( - ExprWrapper { - expr: Arc::new(e1.clone()), - }, - Some(ExprWrapper { - expr: Arc::new(e2.clone()), - }), - ) - }) - .collect(), - } -} - #[cfg(test)] mod tests { use super::*; @@ -5343,7 +5332,7 @@ mod tests { 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]", + "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); @@ -5376,7 +5365,7 @@ mod tests { OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "CoalescePartitionsExec", + "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", ]; @@ -5424,8 +5413,8 @@ mod tests { 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", + "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); @@ -5499,9 +5488,9 @@ mod tests { )?); 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", + "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); @@ -5510,9 +5499,9 @@ mod tests { 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", + "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); @@ -5617,9 +5606,9 @@ mod tests { )?); 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", + "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); @@ -5628,9 +5617,9 @@ mod tests { 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", + "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); @@ -5703,9 +5692,9 @@ mod tests { )?); 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", + "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); @@ -5715,9 +5704,9 @@ mod tests { // 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", + "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); @@ -5736,8 +5725,8 @@ mod tests { // 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", + "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); @@ -5778,8 +5767,8 @@ mod tests { 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", + "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" ]; @@ -5829,8 +5818,8 @@ mod tests { 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]", + "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); @@ -5879,8 +5868,8 @@ mod tests { 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]", + "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); @@ -5904,10 +5893,10 @@ mod tests { 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", + "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); @@ -5915,10 +5904,10 @@ mod tests { let after_optimize = OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " UnionExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", + " UnionExec", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false" ]; @@ -5997,12 +5986,12 @@ mod tests { 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]", + "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); @@ -6011,10 +6000,10 @@ mod tests { 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]", + "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); @@ -6127,13 +6116,13 @@ mod tests { 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]", + "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); @@ -6142,15 +6131,15 @@ mod tests { 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]", + "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); @@ -6264,13 +6253,13 @@ mod tests { 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]", + "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); @@ -6279,15 +6268,15 @@ mod tests { 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]", + "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); diff --git a/datafusion/physical-expr-common/src/expressions/column.rs b/datafusion/physical-expr-common/src/expressions/column.rs index 3c1b45d03728..276ecab6f084 100644 --- a/datafusion/physical-expr-common/src/expressions/column.rs +++ b/datafusion/physical-expr-common/src/expressions/column.rs @@ -21,16 +21,15 @@ 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, }; -use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{internal_err, Result}; use datafusion_expr::ColumnarValue; -use crate::physical_expr::{down_cast_any_ref, ExprMapping, PhysicalExpr}; - /// Represents the column at a given index in a RecordBatch #[derive(Debug, Hash, PartialEq, Eq, Clone)] pub struct Column { @@ -136,36 +135,3 @@ impl Column { pub fn col(name: &str, schema: &Schema) -> Result> { Ok(Arc::new(Column::new_with_schema(name, schema)?)) } - -pub fn update_expression( - expr: Arc, - map: &ExprMapping, -) -> Option> { - if let Some(new_expr) = map.get(expr.clone()) { - new_expr - } else { - update_columns(expr, map) - } -} - -fn update_columns( - expr: Arc, - map: &ExprMapping, -) -> Option> { - if expr.as_any().downcast_ref::().is_some() { - map.get(expr.clone()).unwrap_or(Some(expr)) - } else { - Some( - expr.transform_up(|e: Arc| { - if e.as_any().downcast_ref::().is_some() { - if let Some(Some(updated)) = map.get(e.clone()) { - return Ok(Transformed::yes(updated)); - } - } - Ok(Transformed::no(e)) - }) - .unwrap() - .data, - ) - } -} diff --git a/datafusion/physical-expr-common/src/physical_expr.rs b/datafusion/physical-expr-common/src/physical_expr.rs index 1dbb4356d6ea..c7a19e1a0bf4 100644 --- a/datafusion/physical-expr-common/src/physical_expr.rs +++ b/datafusion/physical-expr-common/src/physical_expr.rs @@ -20,18 +20,21 @@ 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::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::ColumnarValue; -use indexmap::IndexMap; -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` @@ -211,51 +214,94 @@ pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { } } -#[derive(Debug)] -pub struct ExprMapping { - pub map: IndexMap>, -} +/// Wrapper struct for `Arc` to use them as keys in a hash map. +#[derive(Debug, Clone)] +pub struct ExprWrapper(pub Arc); -impl ExprMapping { - pub fn insert( - &mut self, - key: Arc, - value: Option>, - ) -> Option>> { - self.map - .insert( - ExprWrapper { expr: key }, - value.map(|v| ExprWrapper { expr: v }), - ) - .map(|wrapper| wrapper.map(|w| w.expr)) +impl PartialEq for ExprWrapper { + fn eq(&self, other: &Self) -> bool { + self.0.eq(&other.0) } +} - pub fn get( - &self, - key: Arc, - ) -> Option>> { - let key_wrapper = ExprWrapper { expr: key }; - self.map - .get(&key_wrapper) - .map(|value| value.as_ref().map(|v| v.expr.clone())) +impl Eq for ExprWrapper {} + +impl Hash for ExprWrapper { + fn hash(&self, state: &mut H) { + self.0.hash(state); } } #[derive(Debug)] -pub struct ExprWrapper { - pub expr: Arc, +pub struct ExprMapping { + map: IndexMap>>, } -impl PartialEq for ExprWrapper { - fn eq(&self, other: &ExprWrapper) -> bool { - self.expr.eq(&other.expr) - } +#[derive(Clone, Debug)] +pub enum ExprMappingValue { + Present(Arc), + Deleted, + Absent, } -impl Eq for ExprWrapper {} +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 } + } -impl Hash for ExprWrapper { - fn hash(&self, state: &mut H) { - self.expr.hash(state); + 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/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index bd644fde237f..6a4daeaa83ba 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, @@ -1274,24 +1274,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; diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 488fdee52732..55ebd9ed8c44 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -84,9 +84,7 @@ pub use case::{case, CaseExpr}; pub use cast::{cast, cast_with_options, CastExpr}; pub use column::UnKnownColumn; pub use datafusion_expr::utils::format_state_name; -pub use datafusion_physical_expr_common::expressions::column::{ - col, update_expression, Column, -}; +pub use datafusion_physical_expr_common::expressions::column::{col, Column}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e5647d06dd74..bf4c14390e44 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -47,9 +47,7 @@ pub use physical_expr::{ PhysicalExprRef, }; -pub use datafusion_physical_expr_common::physical_expr::{ - ExprMapping, ExprWrapper, 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 e77041789b1b..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; @@ -34,7 +34,6 @@ use datafusion_expr::window_state::{ }; use datafusion_expr::{Accumulator, PartitionEvaluator, WindowFrame, WindowFrameBound}; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use indexmap::IndexMap; /// Common trait for [window function] implementations diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b0fb6d011aff..04043061270f 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -37,18 +37,17 @@ 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 datafusion_physical_expr_common::expressions::column::update_expression; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -92,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"); } @@ -298,15 +294,11 @@ impl ExecutionPlan for FilterExec { self: Arc, map: &ExprMapping, ) -> Result>> { - let Some(new_predicate) = update_expression(self.predicate.clone(), map) else { - return Err(DataFusionError::Internal( - "Filter predicate cannot be empty".to_string(), - )); + let Some(new_predicate) = map.update_expression(self.predicate.clone()) else { + return internal_err!("Filter predicate cannot be empty"); }; - Ok(Some(Arc::new(FilterExec::try_new( - new_predicate, - self.input.clone(), - )?))) + FilterExec::try_new(new_predicate, self.input.clone()) + .map(|e| Some(Arc::new(e) as _)) } } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 9aef0c303c25..737db312ae76 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -34,10 +34,10 @@ use datafusion_common::utils::DataPtr; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ - EquivalenceProperties, LexOrdering, PhysicalSortExpr, PhysicalSortRequirement, + EquivalenceProperties, ExprMapping, LexOrdering, PhysicalSortExpr, + PhysicalSortRequirement, }; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::TryStreamExt; use tokio::task::JoinSet; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 79a77894fe4f..1e7cb3dcfcac 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -46,10 +46,10 @@ 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 datafusion_physical_expr_common::expressions::column::update_expression; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use hashbrown::HashMap; @@ -679,17 +679,15 @@ impl ExecutionPlan for RepartitionExec { { let updated_exprs = exprs .iter() - .map(|expr| update_expression(expr.clone(), map)) + .map(|expr| map.update_expression(expr.clone())) .collect::>(); Partitioning::Hash(updated_exprs.into_iter().flatten().collect(), *size) } else { self.partitioning.clone() }; - Ok(Some(Arc::new(RepartitionExec::try_new( - self.input.clone(), - new_partitioning, - )?))) + RepartitionExec::try_new(self.input.clone(), new_partitioning) + .map(|e| Some(Arc::new(e) as _)) } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index cfdf1ce2bbd3..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,10 +56,8 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; +use datafusion_physical_expr::{ExprMapping, LexOrdering, PhysicalExpr}; -use datafusion_physical_expr_common::expressions::column::update_expression; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; use tokio::sync::mpsc::Sender; @@ -342,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); @@ -728,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(()) } @@ -1012,12 +1007,11 @@ impl ExecutionPlan for SortExec { .expr() .iter() .filter_map(|sort_expr| { - update_expression(sort_expr.expr.clone(), map).map(|new_sort_expr| { - PhysicalSortExpr { + map.update_expression(sort_expr.expr.clone()) + .map(|new_sort_expr| PhysicalSortExpr { expr: new_sort_expr, options: sort_expr.options, - } - }) + }) }) .collect::>(); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index dcd814858a17..68232d61b0f2 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -32,10 +32,8 @@ use crate::{ use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::{ExprMapping, PhysicalExpr, PhysicalSortRequirement}; -use datafusion_physical_expr_common::expressions::column::update_expression; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use log::{debug, trace}; /// Sort preserving merge execution plan @@ -277,12 +275,11 @@ impl ExecutionPlan for SortPreservingMergeExec { .expr .iter() .filter_map(|sort_expr| { - update_expression(sort_expr.expr.clone(), map).map(|new_sort_expr| { - PhysicalSortExpr { + map.update_expression(sort_expr.expr.clone()) + .map(|new_sort_expr| PhysicalSortExpr { expr: new_sort_expr, options: sort_expr.options, - } - }) + }) }) .collect::>(); 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 5d8e5b0f63dc..2c9b3d059516 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -51,18 +51,18 @@ 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 datafusion_physical_expr_common::expressions::column::update_expression; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; @@ -361,15 +361,16 @@ impl ExecutionPlan for BoundedWindowAggExec { let new_keys = self .partition_keys .iter() - .filter_map(|key| update_expression(key.clone(), map)) + .filter_map(|key| map.update_expression(key.clone())) .collect(); - Ok(Some(Arc::new(BoundedWindowAggExec::try_new( + BoundedWindowAggExec::try_new( new_window_exprs, self.input.clone(), new_keys, self.input_order_mode.clone(), - )?))) + ) + .map(|e| Some(Arc::new(e) as _)) } } @@ -1195,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 1550549c964b..a0997a6e66d7 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -44,10 +44,8 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{ExprMapping, PhysicalSortRequirement}; -use datafusion_physical_expr_common::expressions::column::update_expression; -use datafusion_physical_expr_common::physical_expr::ExprMapping; use futures::{ready, Stream, StreamExt}; /// Window execution plan @@ -292,14 +290,11 @@ impl ExecutionPlan for WindowAggExec { let new_keys = self .partition_keys .iter() - .filter_map(|key| update_expression(key.clone(), map)) + .filter_map(|key| map.update_expression(key.clone())) .collect(); - Ok(Some(Arc::new(WindowAggExec::try_new( - new_window_exprs, - self.input.clone(), - new_keys, - )?))) + WindowAggExec::try_new(new_window_exprs, self.input.clone(), new_keys) + .map(|e| Some(Arc::new(e) as _)) } } From 2c7af01aeb32faa1625f4479828cde5291fa0eb6 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 6 May 2024 16:28:22 +0300 Subject: [PATCH 80/85] Use error macros --- .../physical_optimizer/optimize_projections.rs | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 2e99e1feaf5b..c5f3999d0638 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -67,7 +67,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; -use datafusion_common::{JoinSide, JoinType}; +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; @@ -672,8 +672,9 @@ impl ProjectionOptimizer { let Some(new_plan) = plan.update_expressions(&ExprMapping::new(schema_mapping.clone()))? else { - return Err(datafusion_common::DataFusionError::Internal( - "Plans implementing expressions() must also implement update_expressions()".to_string())); + return internal_err!( + "Plans implementing expressions() must also implement update_expressions()" + ); }; self = ProjectionOptimizer { @@ -2159,8 +2160,9 @@ impl ProjectionOptimizer { .clone() .update_expressions(&ExprMapping::new(schema_mapping.clone()))? else { - return Err(datafusion_common::DataFusionError::Internal( - "Plans implementing expressions() must also implement update_expressions()".to_string())); + return internal_err!( + "Plans implementing expressions() must also implement update_expressions()" + ); }; let required_columns = collect_columns_in_plan_schema(&plan); self = ProjectionOptimizer { @@ -4028,9 +4030,7 @@ fn rewrite_filter( ) -> Result> { let map = ExprMapping::new(mapping.clone()); let Some(new_expr) = map.update_expression(predicate.clone()) else { - return Err(datafusion_common::DataFusionError::Internal( - "Filter predicate cannot be rewritten".to_string(), - )); + return internal_err!("Filter predicate cannot be rewritten"); }; FilterExec::try_new(new_expr, input_plan).map(|plan| Arc::new(plan) as _) } From c5d8d974027f27b900b06c522b67be6d2feeb1f4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 6 May 2024 16:41:00 +0300 Subject: [PATCH 81/85] Simplify try_projection_insertion --- .../optimize_projections.rs | 27 +++++++++---------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index c5f3999d0638..3627429bb297 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -574,42 +574,42 @@ impl ProjectionOptimizer { // These plans preserve the input schema, and do not add new requirements. if is_plan_schema_agnostic(&plan) { - self = self.try_insert_below_schema_agnostic()?; + self.try_insert_below_schema_agnostic() } // ------------------------------------------------------------------------ // These plans also preserve the input schema, but may extend requirements. else if is_plan_requirement_extender(&plan) { - self = self.try_insert_below_req_extender()?; + self.try_insert_below_req_extender() } // ------------------------------------------------------------------------ // Preserves schema and do not change requirements, but have multi-child. else if plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some() { - self = self.try_insert_below_union()?; + self.try_insert_below_union() } // ------------------------------------------------------------------------ // Concatenates schemas and do not change requirements. else if let Some(cj) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_cross_join(cj)? + self.try_insert_below_cross_join(cj) } // ------------------------------------------------------------------------ // Joins and aggregations require special attention. else if let Some(hj) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_hash_join(hj)? + self.try_insert_below_hash_join(hj) } else if let Some(nlj) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_nested_loop_join(nlj)? + self.try_insert_below_nested_loop_join(nlj) } else if let Some(smj) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_sort_merge_join(smj)? + self.try_insert_below_sort_merge_join(smj) } else if let Some(shj) = plan.as_any().downcast_ref::() { - self = self.try_insert_below_symmetric_hash_join(shj)? + 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 = self.try_insert_below_aggregate(agg)? + self.try_insert_below_aggregate(agg) } else if plan.as_any().downcast_ref::().is_some() || plan .as_any() @@ -617,23 +617,22 @@ impl ProjectionOptimizer { .is_some() { let window_exprs = collect_window_expressions(&self.plan); - self = match self.try_insert_below_window_execs(window_exprs)? { - optimized if optimized.transformed => optimized.data, + 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, ); - return Ok(no_change.data); + Ok(no_change.data) } } } else { self.children_nodes.iter_mut().for_each(|c| { c.required_columns = collect_columns_in_plan_schema(&c.plan) }); - return Ok(self); + Ok(self) } - Ok(self) } fn try_insert_below_schema_agnostic(mut self) -> Result { From 4b91806b96c09c7f480344a8331b5d6302cb4fc0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 15 May 2024 13:51:53 +0300 Subject: [PATCH 82/85] Update optimize_projections.rs --- .../optimize_projections.rs | 224 +++++++++--------- 1 file changed, 112 insertions(+), 112 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 3627429bb297..9c8ff1f2f4b7 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -466,26 +466,39 @@ impl ProjectionOptimizer { // (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().downcast_ref::().is_some() - || expr.as_any().downcast_ref::().is_some() + expr.as_any().is::() || expr.as_any().is::() }) { return Ok(Transformed::no(self)); } - // New child of the projection is its input child. - let updated_projection = self - .plan - .with_new_children(self.children_nodes[0].plan.children())?; - // The child of the projection is now parent of the projection. - self.plan = self.children_nodes[0] - .plan - .clone() - .with_new_children(vec![updated_projection.clone()])?; + 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; - self.children_nodes[0].plan = updated_projection; // Move the requirements without change. - self.children_nodes[0].required_columns = mem::take(&mut self.required_columns); + 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)) } @@ -569,28 +582,22 @@ impl ProjectionOptimizer { /// When this function returns and recursion on the node finishes, the upper node plans /// are rewritten according to this mapping. This function also updates the parent /// requirements and extends them with self requirements before inserting them to its child(ren). - fn try_projection_insertion(mut self) -> Result { + fn try_projection_insertion(self) -> Result { let plan = self.plan.clone(); - // These plans preserve the input schema, and do not add new requirements. if is_plan_schema_agnostic(&plan) { self.try_insert_below_schema_agnostic() - } - // ------------------------------------------------------------------------ - // These plans also preserve the input schema, but may extend requirements. - else if is_plan_requirement_extender(&plan) { + } else if is_plan_requirement_extender(&plan) { self.try_insert_below_req_extender() + } else { + self.try_insert_below_custom_plans() } - // ------------------------------------------------------------------------ - // Preserves schema and do not change requirements, but have multi-child. - else if plan.as_any().downcast_ref::().is_some() - || plan.as_any().downcast_ref::().is_some() - { - self.try_insert_below_union() - } - // ------------------------------------------------------------------------ + } + + fn try_insert_below_custom_plans(mut self) -> Result { + let plan = self.plan.clone(); // Concatenates schemas and do not change requirements. - else if let Some(cj) = plan.as_any().downcast_ref::() { + if let Some(cj) = plan.as_any().downcast_ref::() { self.try_insert_below_cross_join(cj) } // ------------------------------------------------------------------------ @@ -610,11 +617,8 @@ impl ProjectionOptimizer { return Ok(self); } self.try_insert_below_aggregate(agg) - } else if plan.as_any().downcast_ref::().is_some() - || plan - .as_any() - .downcast_ref::() - .is_some() + } 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)? { @@ -639,16 +643,22 @@ impl ProjectionOptimizer { let plan = self.plan.clone(); let requirement_map = analyze_requirements(&self); if all_columns_required(&requirement_map) { - self.children_nodes[0].required_columns = - mem::take(&mut self.required_columns); + 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_child, schema_mapping) = self.insert_projection(requirement_map)?; + 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: plan.with_new_children(vec![new_child.plan.clone()])?, + plan: new_plan, required_columns: IndexSet::new(), // clear the requirements schema_mapping, - children_nodes: vec![new_child], + children_nodes: new_children, } } Ok(self) @@ -666,7 +676,8 @@ impl ProjectionOptimizer { mem::take(&mut self.required_columns); } else { let plan = self.plan.clone(); - let (new_child, schema_mapping) = self.insert_projection(requirement_map)?; + let (mut new_child, schema_mapping) = + self.insert_projection(requirement_map)?; let Some(new_plan) = plan.update_expressions(&ExprMapping::new(schema_mapping.clone()))? @@ -680,40 +691,7 @@ impl ProjectionOptimizer { plan: new_plan, required_columns: IndexSet::new(), // clear the requirements schema_mapping, - children_nodes: vec![new_child], - } - } - Ok(self) - } - - /// Attempts to insert a projection node below a `UnionExec` node in the query execution plan. - /// - /// This method checks the requirements of the current execution plan to determine if there is any redundancy - /// when it comes to column usage in the context of a `UnionExec`. If all columns are required as per the - /// requirement map, the method updates the required columns for all child nodes accordingly. If not all - /// columns are required, it inserts new projection nodes to optimize the plan, leading to a more efficient - /// execution by reducing unnecessary data processing. - fn try_insert_below_union(mut self) -> Result { - // UnionExec does not change requirements. We can directly check whether there is a redundancy. - let requirement_map = 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 plan = self.plan.clone(); - 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, + children_nodes: vec![new_child.swap_remove(0)], } } Ok(self) @@ -2114,7 +2092,7 @@ impl ProjectionOptimizer { ) { if window_exprs.iter().any(|expr| { expr.clone() - .update_expression(&ExprMapping::new(IndexMap::new())) + .update_expression(&ExprMapping::new(self.schema_mapping.clone())) .is_none() }) { self.children_nodes[0].required_columns = self @@ -2198,29 +2176,48 @@ impl ProjectionOptimizer { fn insert_projection( self, requirement_map: ColumnRequirements, - ) -> Result<(Self, IndexMap)> { + ) -> 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_projection = Arc::new(ProjectionExec::try_new( - projected_exprs, - self.plan.children()[0].clone(), - )?) as _; + 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_requirements = collect_columns_in_plan_schema(&inserted_projection); let new_mapping = calculate_column_mapping(&self.required_columns, &unused_columns); - let result = ProjectionOptimizer { - plan: inserted_projection, - required_columns: new_requirements, - schema_mapping: IndexMap::new(), - children_nodes: self.children_nodes, - }; - Ok((result, new_mapping)) + 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. @@ -3091,17 +3088,17 @@ impl PhysicalOptimizerRule for OptimizeProjections { fn is_plan_schema_determinant(plan: &Arc) -> bool { let plan_any = plan.as_any(); - plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() - | plan_any.downcast_ref::().is_some() + 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::() } /// Given a plan, the function returns the closest node to the root which updates the schema. @@ -4439,25 +4436,26 @@ fn rewrite_bounded_window_aggregate( /// the order with projections without any further adaptation. fn is_plan_schema_agnostic(plan: &Arc) -> bool { let plan_any = plan.as_any(); - plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() + 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.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() - || plan_any.downcast_ref::().is_some() + 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().downcast_ref::().is_some() - || expr.as_any().downcast_ref::().is_some() + expr.as_any().is::() || expr.as_any().is::() } /// Compare the inputs and outputs of the projection. All expressions must be @@ -5903,10 +5901,12 @@ mod tests { let after_optimize = OptimizeProjections::new().optimize(projection, &ConfigOptions::new())?; let expected = [ - "ProjectionExec: expr=[c@2 as c, a@0 as new_a, b@1 as b]", - " UnionExec", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c], has_header=false", + "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" ]; From d2091938107d2862a19105320c1d9d434251ab2e Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Fri, 17 May 2024 10:43:44 +0300 Subject: [PATCH 83/85] Fix the bug --- .../optimize_projections.rs | 40 +- datafusion/sqllogictest/test_files/cte.slt | 792 ------------------ .../sqllogictest/test_files/explain.slt | 2 +- datafusion/sqllogictest/test_files/window.slt | 4 +- 4 files changed, 25 insertions(+), 813 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimize_projections.rs b/datafusion/core/src/physical_optimizer/optimize_projections.rs index 9c8ff1f2f4b7..764a82e99b73 100644 --- a/datafusion/core/src/physical_optimizer/optimize_projections.rs +++ b/datafusion/core/src/physical_optimizer/optimize_projections.rs @@ -89,6 +89,7 @@ use datafusion_physical_plan::joins::{ 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; @@ -3089,16 +3090,18 @@ 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_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. @@ -3245,8 +3248,9 @@ fn collect_projection_input_requirements( projection_exprs: &[(Arc, String)], ) -> IndexSet { required_columns - .iter() - .flat_map(|e| collect_columns(&projection_exprs[e.index()].0)) + .into_iter() + .filter_map(|c| projection_exprs.get(c.index()).map(|e| e.0.clone())) + .flat_map(|e| collect_columns(&e)) .collect::>() } @@ -4880,8 +4884,8 @@ mod tests { }; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{ - ColumnarValue, Operator, ScalarFunctionDefinition, ScalarUDF, ScalarUDFImpl, - Signature, Volatility, WindowFrame, + ColumnarValue, Operator, ScalarUDF, ScalarUDFImpl, Signature, Volatility, + WindowFrame, }; use datafusion_physical_expr::expressions::{ rank, BinaryExpr, CaseExpr, CastExpr, Column, Literal, NegativeExpr, RowNumber, @@ -5007,7 +5011,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), + Arc::new(ScalarUDF::new_from_impl(AddOne::new())), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5073,7 +5077,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 5)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), + Arc::new(ScalarUDF::new_from_impl(AddOne::new())), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5142,7 +5146,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f", 4)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), + Arc::new(ScalarUDF::new_from_impl(AddOne::new())), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b", 1)), @@ -5208,7 +5212,7 @@ mod tests { Arc::new(NegativeExpr::new(Arc::new(Column::new("f_new", 5)))), Arc::new(ScalarFunctionExpr::new( "scalar_expr", - ScalarFunctionDefinition::UDF(Arc::new(ScalarUDF::from(AddOne::new()))), + Arc::new(ScalarUDF::new_from_impl(AddOne::new())), vec![ Arc::new(BinaryExpr::new( Arc::new(Column::new("b_new", 1)), diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 39c1364b758f..d3b817fabf6f 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -15,61 +15,6 @@ # specific language governing permissions and limitations # under the License. -query II -select * from (WITH source AS (select 1 as e) SELECT * FROM source) t1, (WITH source AS (select 1 as e) SELECT * FROM source) t2 ----- -1 1 - -# Ensure table aliases can be case sensitive -query I -WITH "T" AS (SELECT 1 a) SELECT "T".* FROM "T" ----- -1 - -# Ensure table aliases can be case sensitive -query TT -EXPLAIN WITH "NUMBERS" AS (SELECT 1 as a, 2 as b, 3 as c) SELECT "NUMBERS".* FROM "NUMBERS" ----- -logical_plan -01)SubqueryAlias: NUMBERS -02)--Projection: Int64(1) AS a, Int64(2) AS b, Int64(3) AS c -03)----EmptyRelation -physical_plan -01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] -02)--PlaceholderRowExec - -# cte_use_same_name_multiple_times -statement error DataFusion error: Error during planning: WITH query name "a" specified more than once -WITH a AS (SELECT 1), a AS (SELECT 2) SELECT * FROM a; - -# Test disabling recursive CTE -statement ok -set datafusion.execution.enable_recursive_ctes = false; - -query error DataFusion error: This feature is not implemented: Recursive CTEs are not enabled -WITH RECURSIVE nodes AS ( - SELECT 1 as id - UNION ALL - SELECT id + 1 as id - FROM nodes - WHERE id < 3 -) SELECT * FROM nodes - -statement ok -set datafusion.execution.enable_recursive_ctes = true; - - -# DISTINCT UNION is not supported -query error DataFusion error: This feature is not implemented: Recursive queries with a distinct 'UNION' \(in which the previous iteration's results will be de\-duplicated\) is not supported -WITH RECURSIVE nodes AS ( - SELECT 1 as id - UNION - SELECT id + 1 as id - FROM nodes - WHERE id < 3 -) SELECT * FROM nodes - - # trivial recursive CTE works query I rowsort WITH RECURSIVE nodes AS ( @@ -91,740 +36,3 @@ SELECT * FROM nodes 7 8 9 - -# explain trivial recursive CTE -query TT -EXPLAIN WITH RECURSIVE nodes AS ( - SELECT 1 as id - UNION ALL - SELECT id + 1 as id - FROM nodes - WHERE id < 10 -) -SELECT * FROM nodes ----- -logical_plan -01)SubqueryAlias: nodes -02)--RecursiveQuery: is_distinct=false -03)----Projection: Int64(1) AS id -04)------EmptyRelation -05)----Projection: nodes.id + Int64(1) AS id -06)------Filter: nodes.id < Int64(10) -07)--------TableScan: nodes -physical_plan -01)RecursiveQueryExec: name=nodes, is_distinct=false -02)--ProjectionExec: expr=[1 as id] -03)----PlaceholderRowExec -04)--CoalescePartitionsExec -05)----ProjectionExec: expr=[id@0 + 1 as id] -06)------CoalesceBatchesExec: target_batch_size=8192 -07)--------FilterExec: id@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)------------WorkTableExec: name=nodes - -# setup -statement ok -CREATE EXTERNAL TABLE balance STORED as CSV LOCATION '../core/tests/data/recursive_cte/balance.csv' OPTIONS ('format.has_header' 'true'); - -# setup -statement ok -CREATE EXTERNAL TABLE growth STORED as CSV LOCATION '../core/tests/data/recursive_cte/growth.csv' OPTIONS ('format.has_header' 'true'); - -# setup -statement ok -set datafusion.execution.batch_size = 2; - -# recursive CTE with static term derived from table works. -# use explain to ensure that batch size is set to 2. This should produce multiple batches per iteration since the input -# table 'balances' has 4 rows -query TT -EXPLAIN WITH RECURSIVE balances AS ( - SELECT * from balance - UNION ALL - SELECT time + 1 as time, name, account_balance + 10 as account_balance - FROM balances - WHERE time < 10 -) -SELECT * FROM balances -ORDER BY time, name, account_balance ----- -logical_plan -01)Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.account_balance ASC NULLS LAST -02)--SubqueryAlias: balances -03)----RecursiveQuery: is_distinct=false -04)------Projection: balance.time, balance.name, balance.account_balance -05)--------TableScan: balance -06)------Projection: balances.time + Int64(1) AS time, balances.name, balances.account_balance + Int64(10) AS account_balance -07)--------Filter: balances.time < Int64(10) -08)----------TableScan: balances -physical_plan -01)SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] -02)--RecursiveQueryExec: name=balances, is_distinct=false -03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true -04)----CoalescePartitionsExec -05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] -06)--------CoalesceBatchesExec: target_batch_size=2 -07)----------FilterExec: time@0 < 10 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -09)--------------WorkTableExec: name=balances - -# recursive CTE with static term derived from table works -# note that this is run with batch size set to 2. This should produce multiple batches per iteration since the input -# table 'balances' has 4 rows -query ITI -WITH RECURSIVE balances AS ( - SELECT * from balance - UNION ALL - SELECT time + 1 as time, name, account_balance + 10 as account_balance - FROM balances - WHERE time < 10 -) -SELECT * FROM balances -ORDER BY time, name, account_balance ----- -1 John 100 -1 Tim 200 -2 John 110 -2 John 300 -2 Tim 210 -2 Tim 400 -3 John 120 -3 John 310 -3 Tim 220 -3 Tim 410 -4 John 130 -4 John 320 -4 Tim 230 -4 Tim 420 -5 John 140 -5 John 330 -5 Tim 240 -5 Tim 430 -6 John 150 -6 John 340 -6 Tim 250 -6 Tim 440 -7 John 160 -7 John 350 -7 Tim 260 -7 Tim 450 -8 John 170 -8 John 360 -8 Tim 270 -8 Tim 460 -9 John 180 -9 John 370 -9 Tim 280 -9 Tim 470 -10 John 190 -10 John 380 -10 Tim 290 -10 Tim 480 - -# reset batch size to default -statement ok -set datafusion.execution.batch_size = 8182; - -# recursive CTE with recursive join works -query ITI -WITH RECURSIVE balances AS ( - SELECT time as time, name as name, account_balance as account_balance - FROM balance - UNION ALL - SELECT time + 1 as time, balances.name, account_balance + growth.account_growth as account_balance - FROM balances - JOIN growth - ON balances.name = growth.name - WHERE time < 10 -) -SELECT * FROM balances -ORDER BY time, name, account_balance ----- -1 John 100 -1 Tim 200 -2 John 103 -2 John 300 -2 Tim 220 -2 Tim 400 -3 John 106 -3 John 303 -3 Tim 240 -3 Tim 420 -4 John 109 -4 John 306 -4 Tim 260 -4 Tim 440 -5 John 112 -5 John 309 -5 Tim 280 -5 Tim 460 -6 John 115 -6 John 312 -6 Tim 300 -6 Tim 480 -7 John 118 -7 John 315 -7 Tim 320 -7 Tim 500 -8 John 121 -8 John 318 -8 Tim 340 -8 Tim 520 -9 John 124 -9 John 321 -9 Tim 360 -9 Tim 540 -10 John 127 -10 John 324 -10 Tim 380 -10 Tim 560 - -# recursive CTE with aggregations works -query I rowsort -WITH RECURSIVE nodes AS ( - SELECT 1 as id - UNION ALL - SELECT id + 1 as id - FROM nodes - WHERE id < 10 -) -SELECT sum(id) FROM nodes ----- -55 - -# setup -statement ok -CREATE TABLE t(a BIGINT) AS VALUES(1),(2),(3); - -# referencing CTE multiple times does not error -query II rowsort -WITH RECURSIVE my_cte AS ( - SELECT a from t - UNION ALL - SELECT a+2 as a - FROM my_cte - WHERE a<5 -) -SELECT * FROM my_cte t1, my_cte ----- -1 1 -1 2 -1 3 -1 3 -1 4 -1 5 -1 5 -1 6 -2 1 -2 2 -2 3 -2 3 -2 4 -2 5 -2 5 -2 6 -3 1 -3 1 -3 2 -3 2 -3 3 -3 3 -3 3 -3 3 -3 4 -3 4 -3 5 -3 5 -3 5 -3 5 -3 6 -3 6 -4 1 -4 2 -4 3 -4 3 -4 4 -4 5 -4 5 -4 6 -5 1 -5 1 -5 2 -5 2 -5 3 -5 3 -5 3 -5 3 -5 4 -5 4 -5 5 -5 5 -5 5 -5 5 -5 6 -5 6 -6 1 -6 2 -6 3 -6 3 -6 4 -6 5 -6 5 -6 6 - -# CTE within recursive CTE works and does not result in 'index out of bounds: the len is 0 but the index is 0' -query I -WITH RECURSIVE "recursive_cte" AS ( - SELECT 1 as "val" - UNION ALL ( - WITH "sub_cte" AS ( - SELECT - time, - 1 as "val" - FROM - (SELECT DISTINCT "time" FROM "balance") - ) - SELECT - 2 as "val" - FROM - "recursive_cte" - FULL JOIN "sub_cte" ON 1 = 1 - WHERE - "recursive_cte"."val" < 2 - ) -) -SELECT - * -FROM - "recursive_cte"; ----- -1 -2 -2 - -# setup -statement ok -CREATE EXTERNAL TABLE prices STORED as CSV LOCATION '../core/tests/data/recursive_cte/prices.csv' OPTIONS ('format.has_header' 'true'); - -# CTE within window function inside nested CTE works. This test demonstrates using a nested window function to recursively iterate over a column. -query RRII -WITH RECURSIVE "recursive_cte" AS ( - ( - WITH "min_prices_row_num_cte" AS ( - SELECT - MIN("prices"."prices_row_num") AS "prices_row_num" - FROM - "prices" - ), - "min_prices_row_num_cte_second" AS ( - SELECT - MIN("prices"."prices_row_num") AS "prices_row_num_advancement" - FROM - "prices" - WHERE - "prices"."prices_row_num" > ( - SELECT - "prices_row_num" - FROM - "min_prices_row_num_cte" - ) - ) - SELECT - 0.0 AS "beg", - (0.0 + 50) AS "end", - ( - SELECT - "prices_row_num" - FROM - "min_prices_row_num_cte" - ) AS "prices_row_num", - ( - SELECT - "prices_row_num_advancement" - FROM - "min_prices_row_num_cte_second" - ) AS "prices_row_num_advancement" - FROM - "prices" - WHERE - "prices"."prices_row_num" = ( - SELECT - DISTINCT "prices_row_num" - FROM - "min_prices_row_num_cte" - ) - ) - UNION ALL ( - WITH "min_prices_row_num_cte" AS ( - SELECT - "prices"."prices_row_num" AS "prices_row_num", - LEAD("prices"."prices_row_num", 1) OVER ( - ORDER BY "prices_row_num" - ) AS "prices_row_num_advancement" - FROM - ( - SELECT - DISTINCT "prices_row_num" - FROM - "prices" - ) AS "prices" - ) - SELECT - "recursive_cte"."end" AS "beg", - ("recursive_cte"."end" + 50) AS "end", - "min_prices_row_num_cte"."prices_row_num" AS "prices_row_num", - "min_prices_row_num_cte"."prices_row_num_advancement" AS "prices_row_num_advancement" - FROM - "recursive_cte" - FULL JOIN "prices" ON "prices"."prices_row_num" = "recursive_cte"."prices_row_num_advancement" - FULL JOIN "min_prices_row_num_cte" ON "min_prices_row_num_cte"."prices_row_num" = COALESCE( - "prices"."prices_row_num", - "recursive_cte"."prices_row_num_advancement" - ) - WHERE - "recursive_cte"."prices_row_num_advancement" IS NOT NULL - ) -) -SELECT - DISTINCT * -FROM - "recursive_cte" -ORDER BY - "prices_row_num" ASC; ----- -0 50 1 2 -50 100 2 3 -100 150 3 4 -150 200 4 5 -200 250 5 6 -250 300 6 7 -300 350 7 8 -350 400 8 9 -400 450 9 10 -450 500 10 11 -500 550 11 12 -550 600 12 13 -600 650 13 14 -650 700 14 15 -700 750 15 16 -750 800 16 17 -800 850 17 18 -850 900 18 19 -900 950 19 20 -950 1000 20 21 -1000 1050 21 22 -1050 1100 22 23 -1100 1150 23 24 -1150 1200 24 25 -1200 1250 25 26 -1250 1300 26 27 -1300 1350 27 28 -1350 1400 28 29 -1400 1450 29 30 -1450 1500 30 31 -1500 1550 31 32 -1550 1600 32 33 -1600 1650 33 34 -1650 1700 34 35 -1700 1750 35 36 -1750 1800 36 37 -1800 1850 37 38 -1850 1900 38 39 -1900 1950 39 40 -1950 2000 40 41 -2000 2050 41 42 -2050 2100 42 43 -2100 2150 43 44 -2150 2200 44 45 -2200 2250 45 46 -2250 2300 46 47 -2300 2350 47 48 -2350 2400 48 49 -2400 2450 49 50 -2450 2500 50 51 -2500 2550 51 52 -2550 2600 52 53 -2600 2650 53 54 -2650 2700 54 55 -2700 2750 55 56 -2750 2800 56 57 -2800 2850 57 58 -2850 2900 58 59 -2900 2950 59 60 -2950 3000 60 61 -3000 3050 61 62 -3050 3100 62 63 -3100 3150 63 64 -3150 3200 64 65 -3200 3250 65 66 -3250 3300 66 67 -3300 3350 67 68 -3350 3400 68 69 -3400 3450 69 70 -3450 3500 70 71 -3500 3550 71 72 -3550 3600 72 73 -3600 3650 73 74 -3650 3700 74 75 -3700 3750 75 76 -3750 3800 76 77 -3800 3850 77 78 -3850 3900 78 79 -3900 3950 79 80 -3950 4000 80 81 -4000 4050 81 82 -4050 4100 82 83 -4100 4150 83 84 -4150 4200 84 85 -4200 4250 85 86 -4250 4300 86 87 -4300 4350 87 88 -4350 4400 88 89 -4400 4450 89 90 -4450 4500 90 91 -4500 4550 91 92 -4550 4600 92 93 -4600 4650 93 94 -4650 4700 94 95 -4700 4750 95 96 -4750 4800 96 97 -4800 4850 97 98 -4850 4900 98 99 -4900 4950 99 100 -4950 5000 100 NULL - -# setup -statement ok -CREATE EXTERNAL TABLE sales STORED as CSV LOCATION '../core/tests/data/recursive_cte/sales.csv' OPTIONS ('format.has_header' 'true'); - -# setup -statement ok -CREATE EXTERNAL TABLE salespersons STORED as CSV LOCATION '../core/tests/data/recursive_cte/salespersons.csv' OPTIONS ('format.has_header' 'true'); - - -# group by works within recursive cte. This test case demonstrates rolling up a hierarchy of salespeople to their managers. -query III -WITH RECURSIVE region_sales AS ( - -- Anchor member - SELECT - s.salesperson_id AS salesperson_id, - SUM(s.sale_amount) AS amount, - 0 as level - FROM - sales s - GROUP BY - s.salesperson_id - UNION ALL - -- Recursive member - SELECT - sp.manager_id AS salesperson_id, - SUM(rs.amount) AS amount, - MIN(rs.level) + 1 as level - FROM - region_sales rs - INNER JOIN salespersons sp ON rs.salesperson_id = sp.salesperson_id - WHERE sp.manager_id IS NOT NULL - GROUP BY - sp.manager_id -) -SELECT - salesperson_id, - MAX(amount) as amount, - MAX(level) as hierarchy_level -FROM - region_sales -GROUP BY - salesperson_id -ORDER BY - hierarchy_level ASC, salesperson_id ASC; ----- -4 700 0 -5 600 0 -6 500 0 -7 900 0 -2 1300 1 -3 1400 1 -1 2700 2 - -#expect error from recursive CTE with nested recursive terms -query error DataFusion error: This feature is not implemented: Recursive queries cannot be nested -WITH RECURSIVE outer_cte AS ( - SELECT 1 as a - UNION ALL ( - WITH RECURSIVE nested_cte AS ( - SELECT 1 as a - UNION ALL - SELECT a+2 as a - FROM nested_cte where a < 3 - ) - SELECT outer_cte.a +2 - FROM outer_cte JOIN nested_cte USING(a) - WHERE nested_cte.a < 4 - ) -) -SELECT a FROM outer_cte; - -# expect error when recursive CTE is referenced multiple times in the recursive term -query error DataFusion error: This feature is not implemented: Multiple recursive references to the same CTE are not supported -WITH RECURSIVE my_cte AS ( - SELECT 1 as a - UNION ALL - SELECT my_cte.a+2 as a - FROM my_cte join my_cte c2 using(a) - WHERE my_cte.a<5 -) -SELECT a FROM my_cte; - - -# Test issue: https://github.com/apache/datafusion/issues/9680 -query I -WITH RECURSIVE recursive_cte AS ( - SELECT 1 as val - UNION ALL - ( - WITH sub_cte AS ( - SELECT 2 as val - ) - SELECT - 2 as val - FROM recursive_cte - CROSS JOIN sub_cte - WHERE recursive_cte.val < 2 - ) -) -SELECT * FROM recursive_cte; ----- -1 -2 - -# Test issue: https://github.com/apache/datafusion/issues/9680 -# 'recursive_cte' should be on the left of the cross join, as this is the test purpose of the above query. -query TT -explain WITH RECURSIVE recursive_cte AS ( - SELECT 1 as val - UNION ALL - ( - WITH sub_cte AS ( - SELECT 2 as val - ) - SELECT - 2 as val - FROM recursive_cte - CROSS JOIN sub_cte - WHERE recursive_cte.val < 2 - ) -) -SELECT * FROM recursive_cte; ----- -logical_plan -01)SubqueryAlias: recursive_cte -02)--RecursiveQuery: is_distinct=false -03)----Projection: Int64(1) AS val -04)------EmptyRelation -05)----Projection: Int64(2) AS val -06)------CrossJoin: -07)--------Filter: recursive_cte.val < Int64(2) -08)----------TableScan: recursive_cte -09)--------SubqueryAlias: sub_cte -10)----------Projection: Int64(2) AS val -11)------------EmptyRelation -physical_plan -01)RecursiveQueryExec: name=recursive_cte, is_distinct=false -02)--ProjectionExec: expr=[1 as val] -03)----PlaceholderRowExec -04)--ProjectionExec: expr=[2 as val] -05)----CrossJoinExec -06)------CoalescePartitionsExec -07)--------CoalesceBatchesExec: target_batch_size=8182 -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 -query IT rowsort -WITH RECURSIVE my_cte AS( - SELECT 1::int AS a - UNION ALL - SELECT a::bigint+2 FROM my_cte WHERE a<3 -) SELECT *, arrow_typeof(a) FROM my_cte; ----- -1 Int32 -3 Int32 - -# Test issue: https://github.com/apache/datafusion/issues/9794 -# Non-recursive term and recursive term have different number of columns -query error DataFusion error: Error during planning: Non\-recursive term and recursive term must have the same number of columns \(1 != 3\) -WITH RECURSIVE my_cte AS ( - SELECT 1::bigint AS a - UNION ALL - SELECT a+2, 'a','c' FROM my_cte WHERE a<3 -) SELECT * FROM my_cte; - -# Test issue: https://github.com/apache/datafusion/issues/9794 -# Non-recursive term and recursive term have different types, and cannot be casted -query error DataFusion error: Arrow error: Cast error: Cannot cast string 'abc' to value of Int64 type -WITH RECURSIVE my_cte AS ( - SELECT 1 AS a - UNION ALL - SELECT 'abc' FROM my_cte WHERE CAST(a AS text) !='abc' -) SELECT * FROM my_cte; - -# Define a non-recursive CTE in the recursive WITH clause. -# Test issue: https://github.com/apache/datafusion/issues/9804 -query I -WITH RECURSIVE cte AS ( - SELECT a FROM (VALUES(1)) AS t(a) WHERE a > 2 - UNION ALL - SELECT 2 -) SELECT * FROM cte; ----- -2 - -# Define a non-recursive CTE in the recursive WITH clause. -# UNION ALL -query I rowsort -WITH RECURSIVE cte AS ( - SELECT 1 - UNION ALL - SELECT 2 -) SELECT * FROM cte; ----- -1 -2 - -# Define a non-recursive CTE in the recursive WITH clause. -# DISTINCT UNION -query I -WITH RECURSIVE cte AS ( - SELECT 2 - UNION - SELECT 2 -) SELECT * FROM cte; ----- -2 - -# Define a non-recursive CTE in the recursive WITH clause. -# UNION is not present. -query I -WITH RECURSIVE cte AS ( - SELECT 1 -) SELECT * FROM cte; ----- -1 - -# Define a recursive CTE and a non-recursive CTE at the same time. -query II rowsort -WITH RECURSIVE -non_recursive_cte AS ( - SELECT 1 -), -recursive_cte AS ( - SELECT 1 AS a UNION ALL SELECT a+2 FROM recursive_cte WHERE a < 3 -) -SELECT * FROM non_recursive_cte, recursive_cte; ----- -1 1 -1 3 diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index f270013934b0..0a1dc1f54dc4 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -253,7 +253,7 @@ physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after OptimizeProjections SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown 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]:)]] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 6f58e425ae1a..4aee8a984366 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2817,7 +2817,7 @@ physical_plan 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] +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 @@ -2863,7 +2863,7 @@ physical_plan 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] +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 From e782be9822a19325465d8b4efdac0fa47cf619ca Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 21 May 2024 15:04:20 +0300 Subject: [PATCH 84/85] Update cte.slt --- datafusion/sqllogictest/test_files/cte.slt | 792 +++++++++++++++++++++ 1 file changed, 792 insertions(+) diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index d3b817fabf6f..a5d24f9222e1 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -15,6 +15,61 @@ # specific language governing permissions and limitations # under the License. +query II +select * from (WITH source AS (select 1 as e) SELECT * FROM source) t1, (WITH source AS (select 1 as e) SELECT * FROM source) t2 +---- +1 1 + +# Ensure table aliases can be case sensitive +query I +WITH "T" AS (SELECT 1 a) SELECT "T".* FROM "T" +---- +1 + +# Ensure table aliases can be case sensitive +query TT +EXPLAIN WITH "NUMBERS" AS (SELECT 1 as a, 2 as b, 3 as c) SELECT "NUMBERS".* FROM "NUMBERS" +---- +logical_plan +01)SubqueryAlias: NUMBERS +02)--Projection: Int64(1) AS a, Int64(2) AS b, Int64(3) AS c +03)----EmptyRelation +physical_plan +01)ProjectionExec: expr=[1 as a, 2 as b, 3 as c] +02)--PlaceholderRowExec + +# cte_use_same_name_multiple_times +statement error DataFusion error: Error during planning: WITH query name "a" specified more than once +WITH a AS (SELECT 1), a AS (SELECT 2) SELECT * FROM a; + +# Test disabling recursive CTE +statement ok +set datafusion.execution.enable_recursive_ctes = false; + +query error DataFusion error: This feature is not implemented: Recursive CTEs are not enabled +WITH RECURSIVE nodes AS ( + SELECT 1 as id + UNION ALL + SELECT id + 1 as id + FROM nodes + WHERE id < 3 +) SELECT * FROM nodes + +statement ok +set datafusion.execution.enable_recursive_ctes = true; + + +# DISTINCT UNION is not supported +query error DataFusion error: This feature is not implemented: Recursive queries with a distinct 'UNION' \(in which the previous iteration's results will be de\-duplicated\) is not supported +WITH RECURSIVE nodes AS ( + SELECT 1 as id + UNION + SELECT id + 1 as id + FROM nodes + WHERE id < 3 +) SELECT * FROM nodes + + # trivial recursive CTE works query I rowsort WITH RECURSIVE nodes AS ( @@ -36,3 +91,740 @@ SELECT * FROM nodes 7 8 9 + +# explain trivial recursive CTE +query TT +EXPLAIN WITH RECURSIVE nodes AS ( + SELECT 1 as id + UNION ALL + SELECT id + 1 as id + FROM nodes + WHERE id < 10 +) +SELECT * FROM nodes +---- +logical_plan +01)SubqueryAlias: nodes +02)--RecursiveQuery: is_distinct=false +03)----Projection: Int64(1) AS id +04)------EmptyRelation +05)----Projection: nodes.id + Int64(1) AS id +06)------Filter: nodes.id < Int64(10) +07)--------TableScan: nodes +physical_plan +01)RecursiveQueryExec: name=nodes, is_distinct=false +02)--ProjectionExec: expr=[1 as id] +03)----PlaceholderRowExec +04)--CoalescePartitionsExec +05)----ProjectionExec: expr=[id@0 + 1 as id] +06)------CoalesceBatchesExec: target_batch_size=8192 +07)--------FilterExec: id@0 < 10 +08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------WorkTableExec: name=nodes + +# setup +statement ok +CREATE EXTERNAL TABLE balance STORED as CSV LOCATION '../core/tests/data/recursive_cte/balance.csv' OPTIONS ('format.has_header' 'true'); + +# setup +statement ok +CREATE EXTERNAL TABLE growth STORED as CSV LOCATION '../core/tests/data/recursive_cte/growth.csv' OPTIONS ('format.has_header' 'true'); + +# setup +statement ok +set datafusion.execution.batch_size = 2; + +# recursive CTE with static term derived from table works. +# use explain to ensure that batch size is set to 2. This should produce multiple batches per iteration since the input +# table 'balances' has 4 rows +query TT +EXPLAIN WITH RECURSIVE balances AS ( + SELECT * from balance + UNION ALL + SELECT time + 1 as time, name, account_balance + 10 as account_balance + FROM balances + WHERE time < 10 +) +SELECT * FROM balances +ORDER BY time, name, account_balance +---- +logical_plan +01)Sort: balances.time ASC NULLS LAST, balances.name ASC NULLS LAST, balances.account_balance ASC NULLS LAST +02)--SubqueryAlias: balances +03)----RecursiveQuery: is_distinct=false +04)------Projection: balance.time, balance.name, balance.account_balance +05)--------TableScan: balance +06)------Projection: balances.time + Int64(1) AS time, balances.name, balances.account_balance + Int64(10) AS account_balance +07)--------Filter: balances.time < Int64(10) +08)----------TableScan: balances +physical_plan +01)SortExec: expr=[time@0 ASC NULLS LAST,name@1 ASC NULLS LAST,account_balance@2 ASC NULLS LAST], preserve_partitioning=[false] +02)--RecursiveQueryExec: name=balances, is_distinct=false +03)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/recursive_cte/balance.csv]]}, projection=[time, name, account_balance], has_header=true +04)----CoalescePartitionsExec +05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] +06)--------CoalesceBatchesExec: target_batch_size=2 +07)----------FilterExec: time@0 < 10 +08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------------WorkTableExec: name=balances + +# recursive CTE with static term derived from table works +# note that this is run with batch size set to 2. This should produce multiple batches per iteration since the input +# table 'balances' has 4 rows +query ITI +WITH RECURSIVE balances AS ( + SELECT * from balance + UNION ALL + SELECT time + 1 as time, name, account_balance + 10 as account_balance + FROM balances + WHERE time < 10 +) +SELECT * FROM balances +ORDER BY time, name, account_balance +---- +1 John 100 +1 Tim 200 +2 John 110 +2 John 300 +2 Tim 210 +2 Tim 400 +3 John 120 +3 John 310 +3 Tim 220 +3 Tim 410 +4 John 130 +4 John 320 +4 Tim 230 +4 Tim 420 +5 John 140 +5 John 330 +5 Tim 240 +5 Tim 430 +6 John 150 +6 John 340 +6 Tim 250 +6 Tim 440 +7 John 160 +7 John 350 +7 Tim 260 +7 Tim 450 +8 John 170 +8 John 360 +8 Tim 270 +8 Tim 460 +9 John 180 +9 John 370 +9 Tim 280 +9 Tim 470 +10 John 190 +10 John 380 +10 Tim 290 +10 Tim 480 + +# reset batch size to default +statement ok +set datafusion.execution.batch_size = 8182; + +# recursive CTE with recursive join works +query ITI +WITH RECURSIVE balances AS ( + SELECT time as time, name as name, account_balance as account_balance + FROM balance + UNION ALL + SELECT time + 1 as time, balances.name, account_balance + growth.account_growth as account_balance + FROM balances + JOIN growth + ON balances.name = growth.name + WHERE time < 10 +) +SELECT * FROM balances +ORDER BY time, name, account_balance +---- +1 John 100 +1 Tim 200 +2 John 103 +2 John 300 +2 Tim 220 +2 Tim 400 +3 John 106 +3 John 303 +3 Tim 240 +3 Tim 420 +4 John 109 +4 John 306 +4 Tim 260 +4 Tim 440 +5 John 112 +5 John 309 +5 Tim 280 +5 Tim 460 +6 John 115 +6 John 312 +6 Tim 300 +6 Tim 480 +7 John 118 +7 John 315 +7 Tim 320 +7 Tim 500 +8 John 121 +8 John 318 +8 Tim 340 +8 Tim 520 +9 John 124 +9 John 321 +9 Tim 360 +9 Tim 540 +10 John 127 +10 John 324 +10 Tim 380 +10 Tim 560 + +# recursive CTE with aggregations works +query I rowsort +WITH RECURSIVE nodes AS ( + SELECT 1 as id + UNION ALL + SELECT id + 1 as id + FROM nodes + WHERE id < 10 +) +SELECT sum(id) FROM nodes +---- +55 + +# setup +statement ok +CREATE TABLE t(a BIGINT) AS VALUES(1),(2),(3); + +# referencing CTE multiple times does not error +query II rowsort +WITH RECURSIVE my_cte AS ( + SELECT a from t + UNION ALL + SELECT a+2 as a + FROM my_cte + WHERE a<5 +) +SELECT * FROM my_cte t1, my_cte +---- +1 1 +1 2 +1 3 +1 3 +1 4 +1 5 +1 5 +1 6 +2 1 +2 2 +2 3 +2 3 +2 4 +2 5 +2 5 +2 6 +3 1 +3 1 +3 2 +3 2 +3 3 +3 3 +3 3 +3 3 +3 4 +3 4 +3 5 +3 5 +3 5 +3 5 +3 6 +3 6 +4 1 +4 2 +4 3 +4 3 +4 4 +4 5 +4 5 +4 6 +5 1 +5 1 +5 2 +5 2 +5 3 +5 3 +5 3 +5 3 +5 4 +5 4 +5 5 +5 5 +5 5 +5 5 +5 6 +5 6 +6 1 +6 2 +6 3 +6 3 +6 4 +6 5 +6 5 +6 6 + +# CTE within recursive CTE works and does not result in 'index out of bounds: the len is 0 but the index is 0' +query I +WITH RECURSIVE "recursive_cte" AS ( + SELECT 1 as "val" + UNION ALL ( + WITH "sub_cte" AS ( + SELECT + time, + 1 as "val" + FROM + (SELECT DISTINCT "time" FROM "balance") + ) + SELECT + 2 as "val" + FROM + "recursive_cte" + FULL JOIN "sub_cte" ON 1 = 1 + WHERE + "recursive_cte"."val" < 2 + ) +) +SELECT + * +FROM + "recursive_cte"; +---- +1 +2 +2 + +# setup +statement ok +CREATE EXTERNAL TABLE prices STORED as CSV LOCATION '../core/tests/data/recursive_cte/prices.csv' OPTIONS ('format.has_header' 'true'); + +# CTE within window function inside nested CTE works. This test demonstrates using a nested window function to recursively iterate over a column. +query RRII +WITH RECURSIVE "recursive_cte" AS ( + ( + WITH "min_prices_row_num_cte" AS ( + SELECT + MIN("prices"."prices_row_num") AS "prices_row_num" + FROM + "prices" + ), + "min_prices_row_num_cte_second" AS ( + SELECT + MIN("prices"."prices_row_num") AS "prices_row_num_advancement" + FROM + "prices" + WHERE + "prices"."prices_row_num" > ( + SELECT + "prices_row_num" + FROM + "min_prices_row_num_cte" + ) + ) + SELECT + 0.0 AS "beg", + (0.0 + 50) AS "end", + ( + SELECT + "prices_row_num" + FROM + "min_prices_row_num_cte" + ) AS "prices_row_num", + ( + SELECT + "prices_row_num_advancement" + FROM + "min_prices_row_num_cte_second" + ) AS "prices_row_num_advancement" + FROM + "prices" + WHERE + "prices"."prices_row_num" = ( + SELECT + DISTINCT "prices_row_num" + FROM + "min_prices_row_num_cte" + ) + ) + UNION ALL ( + WITH "min_prices_row_num_cte" AS ( + SELECT + "prices"."prices_row_num" AS "prices_row_num", + LEAD("prices"."prices_row_num", 1) OVER ( + ORDER BY "prices_row_num" + ) AS "prices_row_num_advancement" + FROM + ( + SELECT + DISTINCT "prices_row_num" + FROM + "prices" + ) AS "prices" + ) + SELECT + "recursive_cte"."end" AS "beg", + ("recursive_cte"."end" + 50) AS "end", + "min_prices_row_num_cte"."prices_row_num" AS "prices_row_num", + "min_prices_row_num_cte"."prices_row_num_advancement" AS "prices_row_num_advancement" + FROM + "recursive_cte" + FULL JOIN "prices" ON "prices"."prices_row_num" = "recursive_cte"."prices_row_num_advancement" + FULL JOIN "min_prices_row_num_cte" ON "min_prices_row_num_cte"."prices_row_num" = COALESCE( + "prices"."prices_row_num", + "recursive_cte"."prices_row_num_advancement" + ) + WHERE + "recursive_cte"."prices_row_num_advancement" IS NOT NULL + ) +) +SELECT + DISTINCT * +FROM + "recursive_cte" +ORDER BY + "prices_row_num" ASC; +---- +0 50 1 2 +50 100 2 3 +100 150 3 4 +150 200 4 5 +200 250 5 6 +250 300 6 7 +300 350 7 8 +350 400 8 9 +400 450 9 10 +450 500 10 11 +500 550 11 12 +550 600 12 13 +600 650 13 14 +650 700 14 15 +700 750 15 16 +750 800 16 17 +800 850 17 18 +850 900 18 19 +900 950 19 20 +950 1000 20 21 +1000 1050 21 22 +1050 1100 22 23 +1100 1150 23 24 +1150 1200 24 25 +1200 1250 25 26 +1250 1300 26 27 +1300 1350 27 28 +1350 1400 28 29 +1400 1450 29 30 +1450 1500 30 31 +1500 1550 31 32 +1550 1600 32 33 +1600 1650 33 34 +1650 1700 34 35 +1700 1750 35 36 +1750 1800 36 37 +1800 1850 37 38 +1850 1900 38 39 +1900 1950 39 40 +1950 2000 40 41 +2000 2050 41 42 +2050 2100 42 43 +2100 2150 43 44 +2150 2200 44 45 +2200 2250 45 46 +2250 2300 46 47 +2300 2350 47 48 +2350 2400 48 49 +2400 2450 49 50 +2450 2500 50 51 +2500 2550 51 52 +2550 2600 52 53 +2600 2650 53 54 +2650 2700 54 55 +2700 2750 55 56 +2750 2800 56 57 +2800 2850 57 58 +2850 2900 58 59 +2900 2950 59 60 +2950 3000 60 61 +3000 3050 61 62 +3050 3100 62 63 +3100 3150 63 64 +3150 3200 64 65 +3200 3250 65 66 +3250 3300 66 67 +3300 3350 67 68 +3350 3400 68 69 +3400 3450 69 70 +3450 3500 70 71 +3500 3550 71 72 +3550 3600 72 73 +3600 3650 73 74 +3650 3700 74 75 +3700 3750 75 76 +3750 3800 76 77 +3800 3850 77 78 +3850 3900 78 79 +3900 3950 79 80 +3950 4000 80 81 +4000 4050 81 82 +4050 4100 82 83 +4100 4150 83 84 +4150 4200 84 85 +4200 4250 85 86 +4250 4300 86 87 +4300 4350 87 88 +4350 4400 88 89 +4400 4450 89 90 +4450 4500 90 91 +4500 4550 91 92 +4550 4600 92 93 +4600 4650 93 94 +4650 4700 94 95 +4700 4750 95 96 +4750 4800 96 97 +4800 4850 97 98 +4850 4900 98 99 +4900 4950 99 100 +4950 5000 100 NULL + +# setup +statement ok +CREATE EXTERNAL TABLE sales STORED as CSV LOCATION '../core/tests/data/recursive_cte/sales.csv' OPTIONS ('format.has_header' 'true'); + +# setup +statement ok +CREATE EXTERNAL TABLE salespersons STORED as CSV LOCATION '../core/tests/data/recursive_cte/salespersons.csv' OPTIONS ('format.has_header' 'true'); + + +# group by works within recursive cte. This test case demonstrates rolling up a hierarchy of salespeople to their managers. +query III +WITH RECURSIVE region_sales AS ( + -- Anchor member + SELECT + s.salesperson_id AS salesperson_id, + SUM(s.sale_amount) AS amount, + 0 as level + FROM + sales s + GROUP BY + s.salesperson_id + UNION ALL + -- Recursive member + SELECT + sp.manager_id AS salesperson_id, + SUM(rs.amount) AS amount, + MIN(rs.level) + 1 as level + FROM + region_sales rs + INNER JOIN salespersons sp ON rs.salesperson_id = sp.salesperson_id + WHERE sp.manager_id IS NOT NULL + GROUP BY + sp.manager_id +) +SELECT + salesperson_id, + MAX(amount) as amount, + MAX(level) as hierarchy_level +FROM + region_sales +GROUP BY + salesperson_id +ORDER BY + hierarchy_level ASC, salesperson_id ASC; +---- +4 700 0 +5 600 0 +6 500 0 +7 900 0 +2 1300 1 +3 1400 1 +1 2700 2 + +#expect error from recursive CTE with nested recursive terms +query error DataFusion error: This feature is not implemented: Recursive queries cannot be nested +WITH RECURSIVE outer_cte AS ( + SELECT 1 as a + UNION ALL ( + WITH RECURSIVE nested_cte AS ( + SELECT 1 as a + UNION ALL + SELECT a+2 as a + FROM nested_cte where a < 3 + ) + SELECT outer_cte.a +2 + FROM outer_cte JOIN nested_cte USING(a) + WHERE nested_cte.a < 4 + ) +) +SELECT a FROM outer_cte; + +# expect error when recursive CTE is referenced multiple times in the recursive term +query error DataFusion error: This feature is not implemented: Multiple recursive references to the same CTE are not supported +WITH RECURSIVE my_cte AS ( + SELECT 1 as a + UNION ALL + SELECT my_cte.a+2 as a + FROM my_cte join my_cte c2 using(a) + WHERE my_cte.a<5 +) +SELECT a FROM my_cte; + + +# Test issue: https://github.com/apache/datafusion/issues/9680 +query I +WITH RECURSIVE recursive_cte AS ( + SELECT 1 as val + UNION ALL + ( + WITH sub_cte AS ( + SELECT 2 as val + ) + SELECT + 2 as val + FROM recursive_cte + CROSS JOIN sub_cte + WHERE recursive_cte.val < 2 + ) +) +SELECT * FROM recursive_cte; +---- +1 +2 + +# Test issue: https://github.com/apache/datafusion/issues/9680 +# 'recursive_cte' should be on the left of the cross join, as this is the test purpose of the above query. +query TT +explain WITH RECURSIVE recursive_cte AS ( + SELECT 1 as val + UNION ALL + ( + WITH sub_cte AS ( + SELECT 2 as val + ) + SELECT + 2 as val + FROM recursive_cte + CROSS JOIN sub_cte + WHERE recursive_cte.val < 2 + ) +) +SELECT * FROM recursive_cte; +---- +logical_plan +01)SubqueryAlias: recursive_cte +02)--RecursiveQuery: is_distinct=false +03)----Projection: Int64(1) AS val +04)------EmptyRelation +05)----Projection: Int64(2) AS val +06)------CrossJoin: +07)--------Filter: recursive_cte.val < Int64(2) +08)----------TableScan: recursive_cte +09)--------SubqueryAlias: sub_cte +10)----------Projection: Int64(2) AS val +11)------------EmptyRelation +physical_plan +01)RecursiveQueryExec: name=recursive_cte, is_distinct=false +02)--ProjectionExec: expr=[1 as val] +03)----PlaceholderRowExec +04)--ProjectionExec: expr=[2 as val] +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 + +# Test issue: https://github.com/apache/datafusion/issues/9794 +# Non-recursive term and recursive term have different types +query IT rowsort +WITH RECURSIVE my_cte AS( + SELECT 1::int AS a + UNION ALL + SELECT a::bigint+2 FROM my_cte WHERE a<3 +) SELECT *, arrow_typeof(a) FROM my_cte; +---- +1 Int32 +3 Int32 + +# Test issue: https://github.com/apache/datafusion/issues/9794 +# Non-recursive term and recursive term have different number of columns +query error DataFusion error: Error during planning: Non\-recursive term and recursive term must have the same number of columns \(1 != 3\) +WITH RECURSIVE my_cte AS ( + SELECT 1::bigint AS a + UNION ALL + SELECT a+2, 'a','c' FROM my_cte WHERE a<3 +) SELECT * FROM my_cte; + +# Test issue: https://github.com/apache/datafusion/issues/9794 +# Non-recursive term and recursive term have different types, and cannot be casted +query error DataFusion error: Arrow error: Cast error: Cannot cast string 'abc' to value of Int64 type +WITH RECURSIVE my_cte AS ( + SELECT 1 AS a + UNION ALL + SELECT 'abc' FROM my_cte WHERE CAST(a AS text) !='abc' +) SELECT * FROM my_cte; + +# Define a non-recursive CTE in the recursive WITH clause. +# Test issue: https://github.com/apache/datafusion/issues/9804 +query I +WITH RECURSIVE cte AS ( + SELECT a FROM (VALUES(1)) AS t(a) WHERE a > 2 + UNION ALL + SELECT 2 +) SELECT * FROM cte; +---- +2 + +# Define a non-recursive CTE in the recursive WITH clause. +# UNION ALL +query I rowsort +WITH RECURSIVE cte AS ( + SELECT 1 + UNION ALL + SELECT 2 +) SELECT * FROM cte; +---- +1 +2 + +# Define a non-recursive CTE in the recursive WITH clause. +# DISTINCT UNION +query I +WITH RECURSIVE cte AS ( + SELECT 2 + UNION + SELECT 2 +) SELECT * FROM cte; +---- +2 + +# Define a non-recursive CTE in the recursive WITH clause. +# UNION is not present. +query I +WITH RECURSIVE cte AS ( + SELECT 1 +) SELECT * FROM cte; +---- +1 + +# Define a recursive CTE and a non-recursive CTE at the same time. +query II rowsort +WITH RECURSIVE +non_recursive_cte AS ( + SELECT 1 +), +recursive_cte AS ( + SELECT 1 AS a UNION ALL SELECT a+2 FROM recursive_cte WHERE a < 3 +) +SELECT * FROM non_recursive_cte, recursive_cte; +---- +1 1 +1 3 \ No newline at end of file From 21568c29afd85991bd86ee12efd12842bfc6dc5a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Tue, 21 May 2024 15:09:40 +0300 Subject: [PATCH 85/85] Update cte.slt --- datafusion/sqllogictest/test_files/cte.slt | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index a5d24f9222e1..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 @@ -827,4 +827,4 @@ recursive_cte AS ( SELECT * FROM non_recursive_cte, recursive_cte; ---- 1 1 -1 3 \ No newline at end of file +1 3