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