From 7411d6b7ce8f3bc26ffcda11e14e543efe40fd64 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 3 May 2023 10:11:51 +0300 Subject: [PATCH 01/24] Naive test pass i --- .../aggregate_statistics.rs | 12 ++ .../combine_partial_final_agg.rs | 4 + .../physical_optimizer/dist_enforcement.rs | 10 +- .../src/physical_optimizer/repartition.rs | 2 + .../physical_optimizer/sort_enforcement.rs | 1 + .../core/src/physical_plan/aggregates/mod.rs | 55 ++++++++- datafusion/core/src/physical_plan/planner.rs | 106 ++++++++++++++++-- datafusion/core/tests/aggregate_fuzz.rs | 2 + datafusion/core/tests/sql/window.rs | 69 ++++++++++++ datafusion/expr/src/expr.rs | 44 ++++++-- datafusion/expr/src/expr_fn.rs | 12 ++ datafusion/expr/src/tree_node/expr.rs | 17 ++- datafusion/expr/src/udaf.rs | 1 + .../src/analyzer/count_wildcard_rule.rs | 2 + .../optimizer/src/analyzer/type_coercion.rs | 16 ++- .../optimizer/src/common_subexpr_eliminate.rs | 1 + .../optimizer/src/push_down_projection.rs | 5 + .../src/single_distinct_to_groupby.rs | 3 + datafusion/proto/src/generated/prost.rs | 12 ++ .../proto/src/logical_plan/from_proto.rs | 3 + datafusion/proto/src/logical_plan/mod.rs | 4 + datafusion/proto/src/logical_plan/to_proto.rs | 16 ++- .../proto/src/physical_plan/from_proto.rs | 27 +++++ datafusion/proto/src/physical_plan/mod.rs | 23 +++- .../proto/src/physical_plan/to_proto.rs | 27 ++++- datafusion/sql/src/expr/function.rs | 4 +- datafusion/sql/src/expr/mod.rs | 20 ++-- datafusion/sql/src/utils.rs | 10 +- .../substrait/src/logical_plan/consumer.rs | 4 + .../substrait/src/logical_plan/producer.rs | 2 +- 30 files changed, 474 insertions(+), 40 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 4abe3ce0edc4..396e66972f30 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -412,6 +412,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], source, Arc::clone(&schema), )?; @@ -421,6 +422,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -442,6 +444,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], source, Arc::clone(&schema), )?; @@ -451,6 +454,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -471,6 +475,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], source, Arc::clone(&schema), )?; @@ -483,6 +488,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], Arc::new(coalesce), Arc::clone(&schema), )?; @@ -503,6 +509,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], source, Arc::clone(&schema), )?; @@ -515,6 +522,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], Arc::new(coalesce), Arc::clone(&schema), )?; @@ -546,6 +554,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], filter, Arc::clone(&schema), )?; @@ -555,6 +564,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -591,6 +601,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], filter, Arc::clone(&schema), )?; @@ -600,6 +611,7 @@ mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], + vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 0674b6aa31b9..1e45b14d6604 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -73,6 +73,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { group_by: input_group_by, aggr_expr: input_aggr_expr, filter_expr: input_filter_expr, + order_by_expr: input_order_by_expr, input_schema, .. }| { @@ -95,6 +96,7 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { input_group_by.clone(), input_aggr_expr.to_vec(), input_filter_expr.to_vec(), + input_order_by_expr.to_vec(), partial_input.clone(), input_schema.clone(), ) @@ -279,6 +281,7 @@ mod tests { group_by, aggr_expr, vec![], + vec![], input, schema, ) @@ -298,6 +301,7 @@ mod tests { group_by, aggr_expr, vec![], + vec![], input, schema, ) diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index f74a7f2e93e3..6abaae065a24 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -42,7 +42,7 @@ use datafusion_physical_expr::expressions::NoOp; use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ expr_list_eq_strict_order, normalize_expr_with_equivalence_properties, AggregateExpr, - PhysicalExpr, + PhysicalExpr, PhysicalSortExpr, }; use std::sync::Arc; @@ -254,6 +254,7 @@ fn adjust_input_keys_ordering( group_by, aggr_expr, filter_expr, + order_by_expr, input, input_schema, .. @@ -267,6 +268,7 @@ fn adjust_input_keys_ordering( group_by, aggr_expr, filter_expr, + order_by_expr, input.clone(), input_schema, )?), @@ -373,6 +375,7 @@ fn reorder_aggregate_keys( group_by: &PhysicalGroupBy, aggr_expr: &[Arc], filter_expr: &[Option>], + order_by_expr: &[Option], agg_input: Arc, input_schema: &SchemaRef, ) -> Result { @@ -403,6 +406,7 @@ fn reorder_aggregate_keys( group_by, aggr_expr, filter_expr, + order_by_expr, input, input_schema, .. @@ -422,6 +426,7 @@ fn reorder_aggregate_keys( new_partial_group_by, aggr_expr.clone(), filter_expr.clone(), + order_by_expr.clone(), input.clone(), input_schema.clone(), )?)) @@ -453,6 +458,7 @@ fn reorder_aggregate_keys( new_group_by, aggr_expr.to_vec(), filter_expr.to_vec(), + order_by_expr.to_vec(), partial_agg, input_schema.clone(), )?); @@ -1104,12 +1110,14 @@ mod tests { final_grouping, vec![], vec![], + vec![], Arc::new( AggregateExec::try_new( AggregateMode::Partial, group_by, vec![], vec![], + vec![], input, schema.clone(), ) diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index 1db61e379e89..073981ae1ef0 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -478,12 +478,14 @@ mod tests { PhysicalGroupBy::default(), vec![], vec![], + vec![], Arc::new( AggregateExec::try_new( AggregateMode::Partial, PhysicalGroupBy::default(), vec![], vec![], + vec![], input, schema.clone(), ) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 8bb71445a458..3e7ebf2525e2 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2867,6 +2867,7 @@ mod tests { PhysicalGroupBy::default(), vec![], vec![], + vec![], input, schema, ) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 5c77f3b74a65..6c2b881d65ec 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -42,7 +42,7 @@ use datafusion_physical_expr::{ expressions::{Avg, CastExpr, Column, Sum}, normalize_out_expr_with_columns_map, utils::{convert_to_expr, get_indices_of_matching_exprs}, - AggregateExpr, PhysicalExpr, PhysicalSortExpr, + AggregateExpr, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use std::any::Any; use std::collections::HashMap; @@ -228,6 +228,8 @@ pub struct AggregateExec { pub(crate) aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression pub(crate) filter_expr: Vec>>, + /// (ORDER BY clause) expression for each aggregate expression + pub(crate) order_by_expr: Vec>, /// Input plan, could be a partial aggregate or the input to the aggregate pub(crate) input: Arc, /// Schema after the aggregate is applied @@ -243,6 +245,7 @@ pub struct AggregateExec { metrics: ExecutionPlanMetricsSet, /// Stores mode and output ordering information for the `AggregateExec`. aggregation_ordering: Option, + required_input_ordering: Option>, } /// Calculates the working mode for `GROUP BY` queries. @@ -344,6 +347,8 @@ impl AggregateExec { group_by: PhysicalGroupBy, aggr_expr: Vec>, filter_expr: Vec>>, + // TODO: make order_by_expr Option since all aggregators will have same requirement + order_by_expr: Vec>, input: Arc, input_schema: SchemaRef, ) -> Result { @@ -356,6 +361,30 @@ impl AggregateExec { )?; let schema = Arc::new(schema); + println!("aggr_expr: {:?}", aggr_expr); + println!("order_by_expr: {:?}", order_by_expr); + let mut required_input_ordering = None; + if mode == AggregateMode::Partial || mode == AggregateMode::Single { + if let Some(first) = order_by_expr.first() { + if let Some(req) = first { + if group_by.groups.len() == 1 { + let requirement_prefix = output_group_expr_helper(&group_by); + let mut requirement = requirement_prefix + .into_iter() + .map(|expr| PhysicalSortRequirement::new(expr, None)) + .collect::>(); + requirement.push(PhysicalSortRequirement::from(req.clone())); + required_input_ordering = Some(requirement); + } else { + DataFusionError::Plan( + "Cannot run order sensitive aggregation in grouping set queries" + .to_string(), + ); + } + } + } + } + println!("required_input_ordering:{:?}", required_input_ordering); // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); @@ -368,18 +397,25 @@ impl AggregateExec { } let aggregation_ordering = calc_aggregation_ordering(&input, &group_by); + if let Some(aggregation_ordering) = &aggregation_ordering { + println!("ordering: {:?}", aggregation_ordering.ordering); + } else { + println!("ordering: None"); + } Ok(AggregateExec { mode, group_by, aggr_expr, filter_expr, + order_by_expr, input, schema, input_schema, columns_map, metrics: ExecutionPlanMetricsSet::new(), aggregation_ordering, + required_input_ordering, }) } @@ -408,6 +444,11 @@ impl AggregateExec { &self.filter_expr } + /// ORDER BY clause expression for each aggregate expression + pub fn order_by_expr(&self) -> &[Option] { + &self.order_by_expr + } + /// Input plan pub fn input(&self) -> &Arc { &self.input @@ -547,6 +588,10 @@ impl ExecutionPlan for AggregateExec { } } + fn required_input_ordering(&self) -> Vec>> { + vec![self.required_input_ordering.clone()] + } + fn equivalence_properties(&self) -> EquivalenceProperties { let mut new_properties = EquivalenceProperties::new(self.schema()); project_equivalence_properties( @@ -570,6 +615,7 @@ impl ExecutionPlan for AggregateExec { self.group_by.clone(), self.aggr_expr.clone(), self.filter_expr.clone(), + self.order_by_expr.clone(), children[0].clone(), self.input_schema.clone(), )?)) @@ -1131,6 +1177,7 @@ mod tests { grouping_set.clone(), aggregates.clone(), vec![None], + vec![None], input, input_schema.clone(), )?); @@ -1174,6 +1221,7 @@ mod tests { final_grouping_set, aggregates, vec![None], + vec![None], merge, input_schema, )?); @@ -1236,6 +1284,7 @@ mod tests { grouping_set.clone(), aggregates.clone(), vec![None], + vec![None], input, input_schema.clone(), )?); @@ -1269,6 +1318,7 @@ mod tests { final_grouping_set, aggregates, vec![None], + vec![None], merge, input_schema, )?); @@ -1483,6 +1533,7 @@ mod tests { groups, aggregates, vec![None; 3], + vec![None; 3], input.clone(), input_schema.clone(), )?); @@ -1539,6 +1590,7 @@ mod tests { groups.clone(), aggregates.clone(), vec![None], + vec![None], blocking_exec, schema, )?); @@ -1578,6 +1630,7 @@ mod tests { groups, aggregates.clone(), vec![None], + vec![None], blocking_exec, schema, )?); diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index 087545a64420..3ed7fc746476 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -78,6 +78,7 @@ use itertools::Itertools; use log::{debug, trace}; use std::collections::HashMap; use std::fmt::Write; +use std::ops::Deref; use std::sync::Arc; fn create_function_physical_name( @@ -199,12 +200,22 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { args, .. }) => create_function_physical_name(&fun.to_string(), *distinct, args), - Expr::AggregateUDF { fun, args, filter } => { + Expr::AggregateUDF { + fun, + args, + filter, + order_by, + } => { if filter.is_some() { return Err(DataFusionError::Execution( "aggregate expression with filter is not supported".to_string(), )); } + if order_by.is_some() { + return Err(DataFusionError::Execution( + "aggregate expression with order_by is not supported".to_string(), + )); + } let mut names = Vec::with_capacity(args.len()); for e in args { names.push(create_physical_name(e, false)?); @@ -638,6 +649,9 @@ impl DefaultPhysicalPlanner { }) => { // Initially need to perform the aggregate and then merge the partitions let input_exec = self.create_initial_plan(input, session_state).await?; + println!("INPUT PLAN"); + print_plan(&input_exec)?; + println!("INPUT PLAN"); let physical_input_schema = input_exec.schema(); let logical_input_schema = input.as_ref().schema(); @@ -646,7 +660,8 @@ impl DefaultPhysicalPlanner { logical_input_schema, &physical_input_schema, session_state)?; - + println!("aggr_expr: {:?}", aggr_expr); + // let order_by_expr = vec![]; let agg_filter = aggr_expr .iter() .map(|e| { @@ -658,13 +673,20 @@ impl DefaultPhysicalPlanner { ) }) .collect::>>()?; - let (aggregates, filters): (Vec<_>, Vec<_>) = agg_filter.into_iter().unzip(); + let (mut aggregates,mut filters, mut order_bys) = (vec![], vec![], vec![]); + for (aggregate, filter, order_by) in agg_filter.into_iter(){ + aggregates.push(aggregate); + filters.push(filter); + order_bys.push(order_by); + } + // let (aggregates, filters, order_bys): (Vec<_>, Vec<_>, Vec<_>) = agg_filter.into_iter().unzip(); let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), aggregates.clone(), filters.clone(), + order_bys.clone(), input_exec, physical_input_schema.clone(), )?); @@ -701,6 +723,7 @@ impl DefaultPhysicalPlanner { final_grouping_set, aggregates, filters, + order_bys, initial_aggr, physical_input_schema.clone(), )?)) @@ -1318,6 +1341,13 @@ fn merge_grouping_set_physical_expr( )) } +fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) +} + /// Expand and align a CUBE expression. This is a special case of GROUPING SETS /// (see ) fn create_cube_physical_expr( @@ -1579,8 +1609,11 @@ pub fn create_window_expr( ) } -type AggregateExprWithOptionalFilter = - (Arc, Option>); +type AggregateExprWithOptionalFilter = ( + Arc, + Option>, + Option, +); /// Create an aggregate expression with a name from a logical expression pub fn create_aggregate_expr_with_name_and_maybe_filter( @@ -1596,7 +1629,9 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( distinct, args, filter, + order_by, }) => { + println!("order by: {:?}", order_by); let args = args .iter() .map(|e| { @@ -1624,9 +1659,40 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( physical_input_schema, name, ); - Ok((agg_expr?, filter)) + println!("logical_input_schema:{:?}", logical_input_schema.fields()); + println!("physical_input_schema:{:?}", physical_input_schema.fields()); + let order_by = match order_by { + Some(e) => Some(match e.deref() { + Expr::Sort(expr::Sort { + expr, + asc, + nulls_first, + }) => create_physical_sort_expr( + &expr, + logical_input_schema, + physical_input_schema, + SortOptions { + descending: !asc, + nulls_first: *nulls_first, + }, + execution_props, + )?, + _ => { + return Err(DataFusionError::Plan( + "Sort only accepts sort expressions".to_string(), + )) + } + }), + None => None, + }; + Ok((agg_expr?, filter, order_by)) } - Expr::AggregateUDF { fun, args, filter } => { + Expr::AggregateUDF { + fun, + args, + filter, + order_by, + } => { let args = args .iter() .map(|e| { @@ -1648,10 +1714,34 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( )?), None => None, }; + let order_by = match order_by { + Some(e) => Some(match e.deref() { + Expr::Sort(expr::Sort { + expr, + asc, + nulls_first, + }) => create_physical_sort_expr( + &*expr, + logical_input_schema, + physical_input_schema, + SortOptions { + descending: !asc, + nulls_first: *nulls_first, + }, + execution_props, + )?, + _ => { + return Err(DataFusionError::Plan( + "Sort only accepts sort expressions".to_string(), + )) + } + }), + None => None, + }; let agg_expr = udaf::create_aggregate_expr(fun, &args, physical_input_schema, name); - Ok((agg_expr?, filter)) + Ok((agg_expr?, filter, order_by)) } other => Err(DataFusionError::Internal(format!( "Invalid aggregate expression '{other:?}'" diff --git a/datafusion/core/tests/aggregate_fuzz.rs b/datafusion/core/tests/aggregate_fuzz.rs index 14cf46962453..74370049e81f 100644 --- a/datafusion/core/tests/aggregate_fuzz.rs +++ b/datafusion/core/tests/aggregate_fuzz.rs @@ -113,6 +113,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str group_by.clone(), aggregate_expr.clone(), vec![None], + vec![None], running_source, schema.clone(), ) @@ -125,6 +126,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str group_by.clone(), aggregate_expr.clone(), vec![None], + vec![None], usual_source, schema.clone(), ) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 7e7ed031f1ca..14de7ee7875d 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -62,6 +62,7 @@ async fn window_frame_creation_type_checking() -> Result<()> { mod tests { use super::*; + use arrow::util::pretty::print_batches; use datafusion::test_util::{get_test_context, get_test_context2}; #[tokio::test] @@ -522,4 +523,72 @@ mod tests { assert_batches_eq!(expected, &actual); Ok(()) } + + fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) + } + + #[tokio::test] + async fn test_ordering_sensitive_aggregation() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + ctx.sql( + "CREATE TABLE sales_global ( + group_key INT, + sn INT PRIMARY KEY, + ts TIMESTAMP, + currency VARCHAR(3), + amount INT + ) as VALUES + (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), + (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00)", + ) + .await?; + + // let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts + // FROM sales_global AS s + // GROUP BY s.group_key"; + + let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.sn ASC)) AS amounts + FROM sales_global AS s + GROUP BY s.group_key"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // We should produce `BoundedWindowAggExec`s that only works in Sorted mode, since source is finite. + // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. + let expected = { + vec![ + "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.sn ASC NULLS LAST)@1 as amounts]", + " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered", + " SortExec: expr=[group_key@0 ASC NULLS LAST,sn@1 ASC NULLS LAST]", ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + let actual = execute_to_batches(&ctx, sql).await; + + let expected = vec![ + "+-----------+", + "| amounts |", + "+-----------+", + "| [50, 200] |", + "| [75, 100] |", + "+-----------+", + ]; + assert_batches_eq!(expected, &actual); + Ok(()) + } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index d89589a30a36..46ed3553a67c 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -177,6 +177,8 @@ pub enum Expr { args: Vec, /// Optional filter applied prior to aggregating filter: Option>, + /// Optional order by applied prior to aggregating + order_by: Option>, }, /// Returns whether the list contains the expr value. InList { @@ -434,6 +436,8 @@ pub struct AggregateFunction { pub distinct: bool, /// Optional filter pub filter: Option>, + /// Optional Ordering + pub order_by: Option>, } impl AggregateFunction { @@ -442,12 +446,14 @@ impl AggregateFunction { args: Vec, distinct: bool, filter: Option>, + order_by: Option>, ) -> Self { Self { fun, args, distinct, filter, + order_by, } } } @@ -959,12 +965,16 @@ impl fmt::Debug for Expr { distinct, ref args, filter, + order_by, .. }) => { fmt_function(f, &fun.to_string(), *distinct, args, true)?; if let Some(fe) = filter { write!(f, " FILTER (WHERE {fe})")?; } + if let Some(ob) = order_by { + write!(f, " ORDER BY {ob}")?; + } Ok(()) } Expr::AggregateUDF { @@ -1313,25 +1323,35 @@ fn create_name(e: &Expr) -> Result { distinct, args, filter, + order_by, }) => { - let name = create_function_name(&fun.to_string(), *distinct, args)?; + let mut name = create_function_name(&fun.to_string(), *distinct, args)?; if let Some(fe) = filter { - Ok(format!("{name} FILTER (WHERE {fe})")) - } else { - Ok(name) - } + name = format!("{name} FILTER (WHERE {fe})"); + }; + if let Some(order_by) = order_by { + name = format!("{name} FILTER (ORDER BY {order_by})"); + }; + Ok(name) } - Expr::AggregateUDF { fun, args, filter } => { + Expr::AggregateUDF { + fun, + args, + filter, + order_by, + } => { let mut names = Vec::with_capacity(args.len()); for e in args { names.push(create_name(e)?); } - let filter = if let Some(fe) = filter { - format!(" FILTER (WHERE {fe})") - } else { - "".to_string() - }; - Ok(format!("{}({}){}", fun.name, names.join(","), filter)) + let mut info = "".to_string(); + if let Some(fe) = filter { + info += &format!(" FILTER (WHERE {fe})"); + } + if let Some(ob) = order_by { + info += &format!(" ORDER BY ({ob})"); + } + Ok(format!("{}({}){}", fun.name, names.join(","), info)) } Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => { diff --git a/datafusion/expr/src/expr_fn.rs b/datafusion/expr/src/expr_fn.rs index a233ce8260f1..20225ca6eaa2 100644 --- a/datafusion/expr/src/expr_fn.rs +++ b/datafusion/expr/src/expr_fn.rs @@ -105,6 +105,7 @@ pub fn min(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -115,6 +116,7 @@ pub fn max(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -125,6 +127,7 @@ pub fn sum(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -135,6 +138,7 @@ pub fn avg(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -145,6 +149,7 @@ pub fn count(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -200,6 +205,7 @@ pub fn count_distinct(expr: Expr) -> Expr { vec![expr], true, None, + None, )) } @@ -261,6 +267,7 @@ pub fn approx_distinct(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -271,6 +278,7 @@ pub fn median(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -281,6 +289,7 @@ pub fn approx_median(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } @@ -291,6 +300,7 @@ pub fn approx_percentile_cont(expr: Expr, percentile: Expr) -> Expr { vec![expr, percentile], false, None, + None, )) } @@ -305,6 +315,7 @@ pub fn approx_percentile_cont_with_weight( vec![expr, weight_expr, percentile], false, None, + None, )) } @@ -374,6 +385,7 @@ pub fn stddev(expr: Expr) -> Expr { vec![expr], false, None, + None, )) } diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index b0a5e31da0f0..924a2999740d 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -96,13 +96,16 @@ impl TreeNode for Expr { } expr_vec } - Expr::AggregateFunction(AggregateFunction { args, filter, .. }) - | Expr::AggregateUDF { args, filter, .. } => { + Expr::AggregateFunction(AggregateFunction { args, filter, order_by, .. }) + | Expr::AggregateUDF { args, filter, order_by, .. } => { let mut expr_vec = args.clone(); if let Some(f) = filter { expr_vec.push(f.as_ref().clone()); } + if let Some(o) = order_by { + expr_vec.push(o.as_ref().clone()); + } expr_vec } @@ -293,11 +296,13 @@ impl TreeNode for Expr { fun, distinct, filter, + order_by, }) => Expr::AggregateFunction(AggregateFunction::new( fun, transform_vec(args, &mut transform)?, distinct, transform_option_box(filter, &mut transform)?, + order_by, )), Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => Expr::GroupingSet(GroupingSet::Rollup( @@ -315,10 +320,16 @@ impl TreeNode for Expr { )) } }, - Expr::AggregateUDF { args, fun, filter } => Expr::AggregateUDF { + Expr::AggregateUDF { + args, + fun, + filter, + order_by, + } => Expr::AggregateUDF { args: transform_vec(args, &mut transform)?, fun, filter: transform_option_box(filter, &mut transform)?, + order_by: transform_option_box(order_by, &mut transform)?, }, Expr::InList { expr, diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 0ecb5280a942..23bc86f83b29 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -90,6 +90,7 @@ impl AggregateUDF { fun: Arc::new(self.clone()), args, filter: None, + order_by: None, } } } diff --git a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs index 6fdb6b33f2cd..a61d21bec126 100644 --- a/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs +++ b/datafusion/optimizer/src/analyzer/count_wildcard_rule.rs @@ -168,12 +168,14 @@ impl TreeNodeRewriter for CountWildcardRewriter { args, distinct, filter, + order_by, }) if args.len() == 1 => match args[0] { Expr::Wildcard => Expr::AggregateFunction(AggregateFunction { fun: aggregate_function::AggregateFunction::Count, args: vec![lit(COUNT_STAR_EXPANSION)], distinct, filter, + order_by, }), _ => old_expr, }, diff --git a/datafusion/optimizer/src/analyzer/type_coercion.rs b/datafusion/optimizer/src/analyzer/type_coercion.rs index 7a346b5b9e5e..fe0a4b709f03 100644 --- a/datafusion/optimizer/src/analyzer/type_coercion.rs +++ b/datafusion/optimizer/src/analyzer/type_coercion.rs @@ -407,6 +407,7 @@ impl TreeNodeRewriter for TypeCoercionRewriter { args, distinct, filter, + order_by, }) => { let new_expr = coerce_agg_exprs_for_signature( &fun, @@ -415,11 +416,16 @@ impl TreeNodeRewriter for TypeCoercionRewriter { &aggregate_function::signature(&fun), )?; let expr = Expr::AggregateFunction(expr::AggregateFunction::new( - fun, new_expr, distinct, filter, + fun, new_expr, distinct, filter, order_by, )); Ok(expr) } - Expr::AggregateUDF { fun, args, filter } => { + Expr::AggregateUDF { + fun, + args, + filter, + order_by, + } => { let new_expr = coerce_arguments_for_signature( args.as_slice(), &self.schema, @@ -429,6 +435,7 @@ impl TreeNodeRewriter for TypeCoercionRewriter { fun, args: new_expr, filter, + order_by, }; Ok(expr) } @@ -905,6 +912,7 @@ mod test { fun: Arc::new(my_avg), args: vec![lit(10i64)], filter: None, + order_by: None, }; let plan = LogicalPlan::Projection(Projection::try_new(vec![udaf], empty)?); let expected = "Projection: MY_AVG(CAST(Int64(10) AS Float64))\n EmptyRelation"; @@ -935,6 +943,7 @@ mod test { fun: Arc::new(my_avg), args: vec![lit("10")], filter: None, + order_by: None, }; let plan = LogicalPlan::Projection(Projection::try_new(vec![udaf], empty)?); let err = assert_analyzed_plan_eq(Arc::new(TypeCoercion::new()), &plan, "") @@ -956,6 +965,7 @@ mod test { vec![lit(12i64)], false, None, + None, )); let plan = LogicalPlan::Projection(Projection::try_new(vec![agg_expr], empty)?); let expected = "Projection: AVG(Int64(12))\n EmptyRelation"; @@ -968,6 +978,7 @@ mod test { vec![col("a")], false, None, + None, )); let plan = LogicalPlan::Projection(Projection::try_new(vec![agg_expr], empty)?); let expected = "Projection: AVG(a)\n EmptyRelation"; @@ -984,6 +995,7 @@ mod test { vec![lit("1")], false, None, + None, )); let err = Projection::try_new(vec![agg_expr], empty).err().unwrap(); assert_eq!( diff --git a/datafusion/optimizer/src/common_subexpr_eliminate.rs b/datafusion/optimizer/src/common_subexpr_eliminate.rs index be7481f38d31..24665cf90177 100644 --- a/datafusion/optimizer/src/common_subexpr_eliminate.rs +++ b/datafusion/optimizer/src/common_subexpr_eliminate.rs @@ -879,6 +879,7 @@ mod test { )), args: vec![inner], filter: None, + order_by: None, }; // test: common aggregates diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index 5c6f10825a86..a32debad0f13 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -70,6 +70,9 @@ impl OptimizerRule for PushDownProjection { for e in agg.aggr_expr.iter().chain(agg.group_expr.iter()) { expr_to_columns(e, &mut required_columns)? } + println!("agg.aggr_expr: {:?}", agg.aggr_expr); + println!("agg.group_expr: {:?}", agg.group_expr); + println!("required columns: {:?}", required_columns); let new_expr = get_expr(&required_columns, agg.input.schema())?; let projection = LogicalPlan::Projection(Projection::try_new( new_expr, @@ -266,6 +269,7 @@ impl OptimizerRule for PushDownProjection { LogicalPlan::Aggregate(agg) => { let mut required_columns = HashSet::new(); exprlist_to_columns(&projection.expr, &mut required_columns)?; + println!("required_columns2:{:?}", required_columns); // Gather all columns needed for expressions in this Aggregate let mut new_aggr_expr = vec![]; for e in agg.aggr_expr.iter() { @@ -1056,6 +1060,7 @@ mod tests { vec![col("b")], false, Some(Box::new(col("c").gt(lit(42)))), + None, )); let plan = LogicalPlanBuilder::from(table_scan) diff --git a/datafusion/optimizer/src/single_distinct_to_groupby.rs b/datafusion/optimizer/src/single_distinct_to_groupby.rs index cee31b5b3352..ba7e89094b0f 100644 --- a/datafusion/optimizer/src/single_distinct_to_groupby.rs +++ b/datafusion/optimizer/src/single_distinct_to_groupby.rs @@ -131,6 +131,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { fun, args, filter, + order_by, .. }) => { // is_single_distinct_agg ensure args.len=1 @@ -144,6 +145,7 @@ impl OptimizerRule for SingleDistinctToGroupBy { vec![col(SINGLE_DISTINCT_ALIAS)], false, // intentional to remove distinct here filter.clone(), + order_by.clone(), ))) } _ => Ok(aggr_expr.clone()), @@ -399,6 +401,7 @@ mod tests { vec![col("b")], true, None, + None, )), ], )? diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index fd3cdc129235..ef83eba2bba2 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -692,6 +692,8 @@ pub struct AggregateExprNode { pub distinct: bool, #[prost(message, optional, boxed, tag = "4")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, boxed, tag = "5")] + pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -702,6 +704,8 @@ pub struct AggregateUdfExprNode { pub args: ::prost::alloc::vec::Vec, #[prost(message, optional, boxed, tag = "3")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, boxed, tag = "4")] + pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1820,6 +1824,12 @@ pub struct MaybeFilter { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct MaybePhysicalSortExpr { + #[prost(message, optional, tag = "1")] + pub expr: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct AggregateExecNode { #[prost(message, repeated, tag = "1")] pub group_expr: ::prost::alloc::vec::Vec, @@ -1842,6 +1852,8 @@ pub struct AggregateExecNode { pub groups: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "10")] pub filter_expr: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "11")] + pub order_by_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index c66fbc0e7dbe..7d8fb5b40088 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -981,6 +981,7 @@ pub fn parse_expr( .collect::, _>>()?, expr.distinct, parse_optional_expr(expr.filter.as_deref(), registry)?.map(Box::new), + parse_optional_expr(expr.order_by.as_deref(), registry)?.map(Box::new), ))) } ExprType::Alias(alias) => Ok(Expr::Alias( @@ -1373,6 +1374,8 @@ pub fn parse_expr( .collect::, Error>>()?, filter: parse_optional_expr(pb.filter.as_deref(), registry)? .map(Box::new), + order_by: parse_optional_expr(pb.order_by.as_deref(), registry)? + .map(Box::new), }) } diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 8789b7302eff..3525e863220c 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -2527,6 +2527,7 @@ mod roundtrip_tests { vec![col("bananas")], false, None, + None, )); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); @@ -2539,6 +2540,7 @@ mod roundtrip_tests { vec![col("bananas")], true, None, + None, )); let ctx = SessionContext::new(); roundtrip_expr_test(test_expr, ctx); @@ -2551,6 +2553,7 @@ mod roundtrip_tests { vec![col("bananas"), lit(0.42_f32)], false, None, + None, )); let ctx = SessionContext::new(); @@ -2608,6 +2611,7 @@ mod roundtrip_tests { fun: Arc::new(dummy_agg.clone()), args: vec![lit(1.0_f64)], filter: Some(Box::new(lit(true))), + order_by: None, }; let ctx = SessionContext::new(); diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index e757f7830b58..c695dd5af2ef 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -616,6 +616,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { ref args, ref distinct, ref filter, + ref order_by, }) => { let aggr_function = match fun { AggregateFunction::ApproxDistinct => { @@ -668,6 +669,10 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { Some(e) => Some(Box::new(e.as_ref().try_into()?)), None => None, }, + order_by: match order_by { + Some(e) => Some(Box::new(e.as_ref().try_into()?)), + None => None, + }, }; Self { expr_type: Some(ExprType::AggregateExpr(Box::new(aggregate_expr))), @@ -703,7 +708,12 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { .collect::, Error>>()?, })), }, - Expr::AggregateUDF { fun, args, filter } => Self { + Expr::AggregateUDF { + fun, + args, + filter, + order_by, + } => Self { expr_type: Some(ExprType::AggregateUdfExpr(Box::new( protobuf::AggregateUdfExprNode { fun_name: fun.name.clone(), @@ -716,6 +726,10 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { Some(e) => Some(Box::new(e.as_ref().try_into()?)), None => None, }, + order_by: match order_by { + Some(e) => Some(Box::new(e.as_ref().try_into()?)), + None => None, + }, }, ))), }, diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 8e9ba1f6055c..fb7ab7328552 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -61,6 +61,33 @@ impl From<&protobuf::PhysicalColumn> for Column { } } +/// Parses a physical sort expression from a protobuf. +/// +/// # Arguments +/// +/// * `proto` - Input proto with physical sort expression node +/// * `registry` - A registry knows how to build logical expressions out of user-defined function' names +/// * `input_schema` - The Arrow schema for the input, used for determining expression data types +/// when performing type coercion. +pub fn parse_physical_sort_expr( + proto: &protobuf::PhysicalSortExprNode, + registry: &dyn FunctionRegistry, + input_schema: &Schema, +) -> Result { + if let Some(expr) = &proto.expr { + let expr = parse_physical_expr(expr.as_ref(), registry, input_schema)?; + let options = SortOptions { + descending: !proto.asc, + nulls_first: proto.nulls_first, + }; + Ok(PhysicalSortExpr { expr, options }) + } else { + Err(DataFusionError::Execution( + "Cannot parse proto to PhysicalSortExpr".to_string(), + )) + } +} + /// Parses a physical expression from a protobuf. /// /// # Arguments diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index cd8950e97bf8..b473b0a52f75 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -54,7 +54,7 @@ use prost::Message; use crate::common::proto_error; use crate::common::{csv_delimiter_to_string, str_to_byte}; use crate::physical_plan::from_proto::{ - parse_physical_expr, parse_protobuf_file_scan_config, + parse_physical_expr, parse_physical_sort_expr, parse_protobuf_file_scan_config, }; use crate::protobuf::physical_aggregate_expr_node::AggregateFunction; use crate::protobuf::physical_expr_node::ExprType; @@ -416,6 +416,16 @@ impl AsExecutionPlan for PhysicalPlanNode { x.transpose() }) .collect::, _>>()?; + let physical_order_by_expr = hash_agg + .order_by_expr + .iter() + .map(|expr| { + let x = expr.expr.as_ref().map(|e| { + parse_physical_sort_expr(e, registry, &physical_schema) + }); + x.transpose() + }) + .collect::, _>>()?; let physical_aggr_expr: Vec> = hash_agg .aggr_expr @@ -473,6 +483,7 @@ impl AsExecutionPlan for PhysicalPlanNode { PhysicalGroupBy::new(group_expr, null_expr, groups), physical_aggr_expr, physical_filter_expr, + physical_order_by_expr, input, Arc::new((&input_schema).try_into()?), )?)) @@ -893,6 +904,12 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| expr.to_owned().try_into()) .collect::>>()?; + let order_by = exec + .order_by_expr() + .iter() + .map(|expr| expr.to_owned().try_into()) + .collect::>>()?; + let agg = exec .aggr_expr() .iter() @@ -942,6 +959,7 @@ impl AsExecutionPlan for PhysicalPlanNode { group_expr_name: group_names, aggr_expr: agg, filter_expr: filter, + order_by_expr: order_by, aggr_expr_name: agg_names, mode: agg_mode as i32, input: Some(Box::new(input)), @@ -1425,6 +1443,7 @@ mod roundtrip_tests { PhysicalGroupBy::new_single(groups.clone()), aggregates.clone(), vec![None], + vec![None], Arc::new(EmptyExec::new(false, schema.clone())), schema, )?)) @@ -1494,6 +1513,7 @@ mod roundtrip_tests { PhysicalGroupBy::new_single(groups.clone()), aggregates.clone(), vec![None], + vec![None], Arc::new(EmptyExec::new(false, schema.clone())), schema, )?), @@ -1707,6 +1727,7 @@ mod roundtrip_tests { PhysicalGroupBy::new_single(groups), aggregates.clone(), vec![None], + vec![None], Arc::new(EmptyExec::new(false, schema.clone())), schema, )?)) diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 9495c841be70..b30c9c2598c5 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -45,7 +45,7 @@ use crate::protobuf; use crate::protobuf::{physical_aggregate_expr_node, PhysicalSortExprNode, ScalarValue}; use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{DateTimeIntervalExpr, GetIndexedFieldExpr}; -use datafusion::physical_expr::ScalarFunctionExpr; +use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion_common::{DataFusionError, Result}; @@ -530,3 +530,28 @@ impl TryFrom>> for protobuf::MaybeFilter { } } } + +impl TryFrom> for protobuf::MaybePhysicalSortExpr { + type Error = DataFusionError; + + fn try_from(sort_expr: Option) -> Result { + match sort_expr { + None => Ok(protobuf::MaybePhysicalSortExpr { expr: None }), + Some(sort_expr) => Ok(protobuf::MaybePhysicalSortExpr { + expr: Some(sort_expr.try_into()?), + }), + } + } +} + +impl TryFrom for protobuf::PhysicalSortExprNode { + type Error = DataFusionError; + + fn try_from(sort_expr: PhysicalSortExpr) -> std::result::Result { + Ok(PhysicalSortExprNode { + expr: Some(Box::new(sort_expr.expr.try_into()?)), + asc: !sort_expr.options.descending, + nulls_first: sort_expr.options.nulls_first, + }) + } +} diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index bf076a2f3068..dd4ec3b924d3 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -111,8 +111,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let distinct = function.distinct; let (fun, args) = self.aggregate_fn_to_expr(fun, function.args, schema, planner_context)?; + /// TODO: make sure that ordering is parsed correctly return Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, + fun, args, distinct, None, None, ))); }; @@ -131,6 +132,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { fun: fm, args, filter: None, + order_by: None, }); } diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 67fe8b1e99db..75e2e9301b1d 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -317,11 +317,15 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { within_group, } = array_agg; - if let Some(order_by) = order_by { - return Err(DataFusionError::NotImplemented(format!( - "ORDER BY not supported in ARRAY_AGG: {order_by}" - ))); - } + let order_by = if let Some(order_by) = order_by { + Some(Box::new(self.order_by_to_sort_expr( + *order_by, + input_schema, + planner_context, + )?)) + } else { + None + }; if let Some(limit) = limit { return Err(DataFusionError::NotImplemented(format!( @@ -337,11 +341,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let args = vec![self.sql_expr_to_logical_expr(*expr, input_schema, planner_context)?]; + // next, aggregate built-ins let fun = AggregateFunction::ArrayAgg; - Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, + fun, args, distinct, None, order_by, ))) } @@ -479,6 +483,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { fun, args, distinct, + order_by, .. }) => Ok(Expr::AggregateFunction(expr::AggregateFunction::new( fun, @@ -489,6 +494,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { schema, planner_context, )?)), + order_by, ))), _ => Err(DataFusionError::Internal( "AggregateExpressionWithFilter expression was not an AggregateFunction" diff --git a/datafusion/sql/src/utils.rs b/datafusion/sql/src/utils.rs index 91cef6d4712e..89494d6efec9 100644 --- a/datafusion/sql/src/utils.rs +++ b/datafusion/sql/src/utils.rs @@ -168,6 +168,7 @@ where args, distinct, filter, + order_by, }) => Ok(Expr::AggregateFunction(AggregateFunction::new( fun.clone(), args.iter() @@ -175,6 +176,7 @@ where .collect::>>()?, *distinct, filter.clone(), + order_by.clone(), ))), Expr::WindowFunction(WindowFunction { fun, @@ -197,13 +199,19 @@ where .collect::>>()?, window_frame.clone(), ))), - Expr::AggregateUDF { fun, args, filter } => Ok(Expr::AggregateUDF { + Expr::AggregateUDF { + fun, + args, + filter, + order_by, + } => Ok(Expr::AggregateUDF { fun: fun.clone(), args: args .iter() .map(|e| clone_with_replacement(e, replacement_fn)) .collect::>>()?, filter: filter.clone(), + order_by: order_by.clone(), }), Expr::Alias(nested_expr, alias_name) => Ok(Expr::Alias( Box::new(clone_with_replacement(nested_expr, replacement_fn)?), diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 2b8ffde4229c..31654e863789 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -278,6 +278,8 @@ pub async fn from_substrait_rel( input.schema(), extensions, filter, + // TODO: Add parsing of order_by also + None, distinct, ) .await @@ -549,6 +551,7 @@ pub async fn from_substrait_agg_func( input_schema: &DFSchema, extensions: &HashMap, filter: Option>, + order_by: Option>, distinct: bool, ) -> Result> { let mut args: Vec = vec![]; @@ -579,6 +582,7 @@ pub async fn from_substrait_agg_func( args, distinct, filter, + order_by, }))) } diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 17d424cea6a0..679b01b4f0eb 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -415,7 +415,7 @@ pub fn to_substrait_agg_measure( ), ) -> Result { match expr { - Expr::AggregateFunction(expr::AggregateFunction { fun, args, distinct, filter }) => { + Expr::AggregateFunction(expr::AggregateFunction { fun, args, distinct, filter, order_by }) => { let mut arguments: Vec = vec![]; for arg in args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, extension_info)?)) }); From 87c397dc4486dd65f0ed30260af0ed8c62c0bc23 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 3 May 2023 15:46:45 +0300 Subject: [PATCH 02/24] Add new tests and simplifications --- .../core/src/physical_plan/aggregates/mod.rs | 84 ++++++--- datafusion/core/src/physical_plan/planner.rs | 144 +++++---------- datafusion/core/tests/sql/window.rs | 173 ++++++++++++++++-- .../optimizer/src/push_down_projection.rs | 4 - datafusion/sql/src/expr/function.rs | 1 - 5 files changed, 263 insertions(+), 143 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 6c2b881d65ec..854f8b8e544e 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -40,9 +40,10 @@ use datafusion_physical_expr::{ aggregate::row_accumulator::RowAccumulator, equivalence::project_equivalence_properties, expressions::{Avg, CastExpr, Column, Sum}, - normalize_out_expr_with_columns_map, + normalize_out_expr_with_columns_map, normalize_sort_expr_with_equivalence_properties, utils::{convert_to_expr, get_indices_of_matching_exprs}, - AggregateExpr, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + AggregateExpr, EquivalentClass, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, }; use std::any::Any; use std::collections::HashMap; @@ -340,6 +341,35 @@ fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec], + eq_properties: &[EquivalentClass], +) -> Result> { + let mut res: Option = None; + for elem in order_by_expr.iter().flatten() { + if let Some(res) = &res { + let res_normalized = normalize_sort_expr_with_equivalence_properties( + res.clone(), + eq_properties, + ); + let elem_normalized = normalize_sort_expr_with_equivalence_properties( + elem.clone(), + eq_properties, + ); + if !res_normalized.eq(&elem_normalized) { + println!("Conflicting requirements"); + return Err(DataFusionError::Plan( + "Conflicting ordering requirements in aggregate functions" + .to_string(), + )); + } + } else { + res = Some(elem.clone()) + } + } + Ok(res) +} + impl AggregateExec { /// Create a new hash aggregate execution plan pub fn try_new( @@ -347,7 +377,6 @@ impl AggregateExec { group_by: PhysicalGroupBy, aggr_expr: Vec>, filter_expr: Vec>>, - // TODO: make order_by_expr Option since all aggregators will have same requirement order_by_expr: Vec>, input: Arc, input_schema: SchemaRef, @@ -361,30 +390,38 @@ impl AggregateExec { )?; let schema = Arc::new(schema); - println!("aggr_expr: {:?}", aggr_expr); - println!("order_by_expr: {:?}", order_by_expr); let mut required_input_ordering = None; if mode == AggregateMode::Partial || mode == AggregateMode::Single { - if let Some(first) = order_by_expr.first() { - if let Some(req) = first { - if group_by.groups.len() == 1 { - let requirement_prefix = output_group_expr_helper(&group_by); - let mut requirement = requirement_prefix - .into_iter() - .map(|expr| PhysicalSortRequirement::new(expr, None)) - .collect::>(); - requirement.push(PhysicalSortRequirement::from(req.clone())); - required_input_ordering = Some(requirement); - } else { - DataFusionError::Plan( - "Cannot run order sensitive aggregation in grouping set queries" - .to_string(), - ); + let requirement = get_finest_requirement( + &order_by_expr, + input.equivalence_properties().classes(), + )?; + if let Some(req) = requirement { + if group_by.groups.len() == 1 { + let requirement_prefix = output_group_expr_helper(&group_by); + let mut requirement = requirement_prefix + .into_iter() + .map(|expr| PhysicalSortRequirement::new(expr, None)) + .collect::>(); + let mut found = false; + for elem in &requirement { + if req.expr.eq(elem.expr()) { + found = true; + break; + } + } + if !found { + requirement.push(PhysicalSortRequirement::from(req)); } + required_input_ordering = Some(requirement); + } else { + return Err(DataFusionError::Plan( + "Cannot run order sensitive aggregation in grouping set queries" + .to_string(), + )); } } } - println!("required_input_ordering:{:?}", required_input_ordering); // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); @@ -397,11 +434,6 @@ impl AggregateExec { } let aggregation_ordering = calc_aggregation_ordering(&input, &group_by); - if let Some(aggregation_ordering) = &aggregation_ordering { - println!("ordering: {:?}", aggregation_ordering.ordering); - } else { - println!("ordering: None"); - } Ok(AggregateExec { mode, diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index 3ed7fc746476..17c5d81703fc 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -78,7 +78,6 @@ use itertools::Itertools; use log::{debug, trace}; use std::collections::HashMap; use std::fmt::Write; -use std::ops::Deref; use std::sync::Arc; fn create_function_physical_name( @@ -649,9 +648,6 @@ impl DefaultPhysicalPlanner { }) => { // Initially need to perform the aggregate and then merge the partitions let input_exec = self.create_initial_plan(input, session_state).await?; - println!("INPUT PLAN"); - print_plan(&input_exec)?; - println!("INPUT PLAN"); let physical_input_schema = input_exec.schema(); let logical_input_schema = input.as_ref().schema(); @@ -660,8 +656,6 @@ impl DefaultPhysicalPlanner { logical_input_schema, &physical_input_schema, session_state)?; - println!("aggr_expr: {:?}", aggr_expr); - // let order_by_expr = vec![]; let agg_filter = aggr_expr .iter() .map(|e| { @@ -679,7 +673,6 @@ impl DefaultPhysicalPlanner { filters.push(filter); order_bys.push(order_by); } - // let (aggregates, filters, order_bys): (Vec<_>, Vec<_>, Vec<_>) = agg_filter.into_iter().unzip(); let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, @@ -847,25 +840,12 @@ impl DefaultPhysicalPlanner { let input_dfschema = input.as_ref().schema(); let sort_expr = expr .iter() - .map(|e| match e { - Expr::Sort(expr::Sort { - expr, - asc, - nulls_first, - }) => create_physical_sort_expr( - expr, - input_dfschema, - &input_schema, - SortOptions { - descending: !*asc, - nulls_first: *nulls_first, - }, - session_state.execution_props(), - ), - _ => Err(DataFusionError::Plan( - "Sort only accepts sort expressions".to_string(), - )), - }) + .map(|e| create_physical_sort_expr( + e, + input_dfschema, + &input_schema, + session_state.execution_props(), + )) .collect::>>()?; let new_sort = SortExec::new(sort_expr, physical_input) .with_fetch(*fetch); @@ -1341,13 +1321,6 @@ fn merge_grouping_set_physical_expr( )) } -fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent().to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) -} - /// Expand and align a CUBE expression. This is a special case of GROUPING SETS /// (see ) fn create_cube_physical_expr( @@ -1544,24 +1517,13 @@ pub fn create_window_expr_with_name( .collect::>>()?; let order_by = order_by .iter() - .map(|e| match e { - Expr::Sort(expr::Sort { - expr, - asc, - nulls_first, - }) => create_physical_sort_expr( - expr, + .map(|e| { + create_physical_sort_expr( + e, logical_input_schema, physical_input_schema, - SortOptions { - descending: !*asc, - nulls_first: *nulls_first, - }, execution_props, - ), - _ => Err(DataFusionError::Plan( - "Sort only accepts sort expressions".to_string(), - )), + ) }) .collect::>>()?; if !is_window_valid(window_frame) { @@ -1631,7 +1593,6 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( filter, order_by, }) => { - println!("order by: {:?}", order_by); let args = args .iter() .map(|e| { @@ -1659,30 +1620,13 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( physical_input_schema, name, ); - println!("logical_input_schema:{:?}", logical_input_schema.fields()); - println!("physical_input_schema:{:?}", physical_input_schema.fields()); let order_by = match order_by { - Some(e) => Some(match e.deref() { - Expr::Sort(expr::Sort { - expr, - asc, - nulls_first, - }) => create_physical_sort_expr( - &expr, - logical_input_schema, - physical_input_schema, - SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - execution_props, - )?, - _ => { - return Err(DataFusionError::Plan( - "Sort only accepts sort expressions".to_string(), - )) - } - }), + Some(e) => Some(create_physical_sort_expr( + e, + logical_input_schema, + physical_input_schema, + execution_props, + )?), None => None, }; Ok((agg_expr?, filter, order_by)) @@ -1715,27 +1659,12 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( None => None, }; let order_by = match order_by { - Some(e) => Some(match e.deref() { - Expr::Sort(expr::Sort { - expr, - asc, - nulls_first, - }) => create_physical_sort_expr( - &*expr, - logical_input_schema, - physical_input_schema, - SortOptions { - descending: !asc, - nulls_first: *nulls_first, - }, - execution_props, - )?, - _ => { - return Err(DataFusionError::Plan( - "Sort only accepts sort expressions".to_string(), - )) - } - }), + Some(e) => Some(create_physical_sort_expr( + e, + logical_input_schema, + physical_input_schema, + execution_props, + )?), None => None, }; @@ -1776,13 +1705,32 @@ pub fn create_physical_sort_expr( e: &Expr, input_dfschema: &DFSchema, input_schema: &Schema, - options: SortOptions, execution_props: &ExecutionProps, ) -> Result { - Ok(PhysicalSortExpr { - expr: create_physical_expr(e, input_dfschema, input_schema, execution_props)?, - options, - }) + match e { + Expr::Sort(expr::Sort { + expr, + asc, + nulls_first, + }) => { + let options = SortOptions { + descending: !asc, + nulls_first: *nulls_first, + }; + Ok(PhysicalSortExpr { + expr: create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?, + options, + }) + } + _ => Err(DataFusionError::Plan( + "Sort only accepts sort expressions".to_string(), + )), + } } impl DefaultPhysicalPlanner { diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 14de7ee7875d..b897df162da9 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -524,13 +524,6 @@ mod tests { Ok(()) } - fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent().to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) - } - #[tokio::test] async fn test_ordering_sensitive_aggregation() -> Result<()> { let config = SessionConfig::new().with_target_partitions(1); @@ -550,11 +543,7 @@ mod tests { ) .await?; - // let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts - // FROM sales_global AS s - // GROUP BY s.group_key"; - - let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.sn ASC)) AS amounts + let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts FROM sales_global AS s GROUP BY s.group_key"; @@ -566,9 +555,10 @@ mod tests { // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. let expected = { vec![ - "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.sn ASC NULLS LAST)@1 as amounts]", + "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts]", " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered", - " SortExec: expr=[group_key@0 ASC NULLS LAST,sn@1 ASC NULLS LAST]", ] + " SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 ASC NULLS LAST]", + ] }; let actual: Vec<&str> = formatted.trim().lines().collect(); @@ -591,4 +581,159 @@ mod tests { assert_batches_eq!(expected, &actual); Ok(()) } + + #[tokio::test] + async fn test_ordering_sensitive_aggregation2() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + ctx.sql( + "CREATE TABLE sales_global ( + group_key INT, + sn INT PRIMARY KEY, + ts TIMESTAMP, + currency VARCHAR(3), + amount INT + ) as VALUES + (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), + (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + ", + ) + .await?; + + let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts + FROM sales_global AS s + GROUP BY s.group_key"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // We should produce `BoundedWindowAggExec`s that only works in Sorted mode, since source is finite. + // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. + let expected = { + vec![ + "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS FIRST)@1 as amounts]", + " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered", + " SortExec: expr=[group_key@0 ASC NULLS LAST]", + ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + let actual = execute_to_batches(&ctx, sql).await; + + let expected = vec![ + "+---------------+", + "| amounts |", + "+---------------+", + "| [50, 200] |", + "| [75, 100, 80] |", + "+---------------+", + ]; + assert_batches_eq!(expected, &actual); + Ok(()) + } + + #[tokio::test] + async fn test_ordering_sensitive_aggregation3() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + ctx.sql( + "CREATE TABLE sales_global ( + group_key INT, + sn INT PRIMARY KEY, + ts TIMESTAMP, + currency VARCHAR(3), + amount INT + ) as VALUES + (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), + (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + ", + ) + .await?; + + let sql = "SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2 + FROM sales_global AS s + GROUP BY s.group_key"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await; + let err_msg = "Conflicting ordering requirements in aggregate functions"; + assert_contains!(physical_plan.err().unwrap().to_string(), err_msg); + Ok(()) + } + + #[tokio::test] + async fn test_ordering_sensitive_aggregation4() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + ctx.sql( + "CREATE TABLE sales_global ( + group_key INT, + sn INT PRIMARY KEY, + ts TIMESTAMP, + currency VARCHAR(3), + amount INT + ) as VALUES + (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), + (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + ", + ) + .await?; + + let sql = "SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.group_key"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // We should produce `BoundedWindowAggExec`s that only works in Sorted mode, since source is finite. + // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. + let expected = { + vec![ + "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1]", + " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered", + " SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 DESC]", + ] + }; + + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + let actual_trim_last = &actual[..actual_len - 1]; + assert_eq!( + expected, actual_trim_last, + "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + let actual = execute_to_batches(&ctx, sql).await; + + let expected = vec![ + "+---------------+------+", + "| amounts | sum1 |", + "+---------------+------+", + "| [200, 50] | 250 |", + "| [100, 80, 75] | 255 |", + "+---------------+------+", + ]; + assert_batches_eq!(expected, &actual); + Ok(()) + } } diff --git a/datafusion/optimizer/src/push_down_projection.rs b/datafusion/optimizer/src/push_down_projection.rs index a32debad0f13..c64dfc578b96 100644 --- a/datafusion/optimizer/src/push_down_projection.rs +++ b/datafusion/optimizer/src/push_down_projection.rs @@ -70,9 +70,6 @@ impl OptimizerRule for PushDownProjection { for e in agg.aggr_expr.iter().chain(agg.group_expr.iter()) { expr_to_columns(e, &mut required_columns)? } - println!("agg.aggr_expr: {:?}", agg.aggr_expr); - println!("agg.group_expr: {:?}", agg.group_expr); - println!("required columns: {:?}", required_columns); let new_expr = get_expr(&required_columns, agg.input.schema())?; let projection = LogicalPlan::Projection(Projection::try_new( new_expr, @@ -269,7 +266,6 @@ impl OptimizerRule for PushDownProjection { LogicalPlan::Aggregate(agg) => { let mut required_columns = HashSet::new(); exprlist_to_columns(&projection.expr, &mut required_columns)?; - println!("required_columns2:{:?}", required_columns); // Gather all columns needed for expressions in this Aggregate let mut new_aggr_expr = vec![]; for e in agg.aggr_expr.iter() { diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index dd4ec3b924d3..4575d0b28a6d 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -111,7 +111,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let distinct = function.distinct; let (fun, args) = self.aggregate_fn_to_expr(fun, function.args, schema, planner_context)?; - /// TODO: make sure that ordering is parsed correctly return Ok(Expr::AggregateFunction(expr::AggregateFunction::new( fun, args, distinct, None, None, ))); From ebd0e5f475d51f177ec9575075b2b31cec901e3a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 4 May 2023 18:13:43 +0300 Subject: [PATCH 03/24] move tests to the .slt file --- datafusion/core/tests/sql/window.rs | 152 ------------------ .../sqllogictests/test_files/aggregate.slt | 108 +++++++++++++ 2 files changed, 108 insertions(+), 152 deletions(-) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index b897df162da9..586561a2de9c 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -526,158 +526,6 @@ mod tests { #[tokio::test] async fn test_ordering_sensitive_aggregation() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - ctx.sql( - "CREATE TABLE sales_global ( - group_key INT, - sn INT PRIMARY KEY, - ts TIMESTAMP, - currency VARCHAR(3), - amount INT - ) as VALUES - (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), - (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00)", - ) - .await?; - - let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts - FROM sales_global AS s - GROUP BY s.group_key"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - let formatted = displayable(physical_plan.as_ref()).indent().to_string(); - // We should produce `BoundedWindowAggExec`s that only works in Sorted mode, since source is finite. - // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. - let expected = { - vec![ - "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts]", - " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered", - " SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 ASC NULLS LAST]", - ] - }; - - let actual: Vec<&str> = formatted.trim().lines().collect(); - let actual_len = actual.len(); - let actual_trim_last = &actual[..actual_len - 1]; - assert_eq!( - expected, actual_trim_last, - "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - let actual = execute_to_batches(&ctx, sql).await; - - let expected = vec![ - "+-----------+", - "| amounts |", - "+-----------+", - "| [50, 200] |", - "| [75, 100] |", - "+-----------+", - ]; - assert_batches_eq!(expected, &actual); - Ok(()) - } - - #[tokio::test] - async fn test_ordering_sensitive_aggregation2() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - ctx.sql( - "CREATE TABLE sales_global ( - group_key INT, - sn INT PRIMARY KEY, - ts TIMESTAMP, - currency VARCHAR(3), - amount INT - ) as VALUES - (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), - (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) - ", - ) - .await?; - - let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts - FROM sales_global AS s - GROUP BY s.group_key"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - let formatted = displayable(physical_plan.as_ref()).indent().to_string(); - // We should produce `BoundedWindowAggExec`s that only works in Sorted mode, since source is finite. - // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. - let expected = { - vec![ - "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS FIRST)@1 as amounts]", - " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered", - " SortExec: expr=[group_key@0 ASC NULLS LAST]", - ] - }; - - let actual: Vec<&str> = formatted.trim().lines().collect(); - let actual_len = actual.len(); - let actual_trim_last = &actual[..actual_len - 1]; - assert_eq!( - expected, actual_trim_last, - "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - let actual = execute_to_batches(&ctx, sql).await; - - let expected = vec![ - "+---------------+", - "| amounts |", - "+---------------+", - "| [50, 200] |", - "| [75, 100, 80] |", - "+---------------+", - ]; - assert_batches_eq!(expected, &actual); - Ok(()) - } - - #[tokio::test] - async fn test_ordering_sensitive_aggregation3() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - ctx.sql( - "CREATE TABLE sales_global ( - group_key INT, - sn INT PRIMARY KEY, - ts TIMESTAMP, - currency VARCHAR(3), - amount INT - ) as VALUES - (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), - (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) - ", - ) - .await?; - - let sql = "SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2 - FROM sales_global AS s - GROUP BY s.group_key"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await; - let err_msg = "Conflicting ordering requirements in aggregate functions"; - assert_contains!(physical_plan.err().unwrap().to_string(), err_msg); - Ok(()) - } - - #[tokio::test] - async fn test_ordering_sensitive_aggregation4() -> Result<()> { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::with_config(config); ctx.sql( diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 9998e766f783..86267743edc6 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1643,3 +1643,111 @@ NULL 1 10.1 10.1 10.1 10.1 0 NULL # Restore the default dialect statement ok set datafusion.sql_parser.dialect = 'Generic'; + +# create a table for testing +statement ok +CREATE TABLE sales_global (group_key INT, + sn INT PRIMARY KEY, + ts TIMESTAMP, + currency VARCHAR(3), + amount INT + ) as VALUES + (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), + (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), + (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + +# test_ordering_sensitive_aggregation +statement ok +set datafusion.execution.target_partitions = 1; + +query TT +EXPLAIN SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts + FROM sales_global AS s + GROUP BY s.group_key +---- +logical_plan +Projection: ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST) AS amounts + Aggregate: groupBy=[[s.group_key]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST]] + SubqueryAlias: s + TableScan: sales_global projection=[group_key, amount] +physical_plan +ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] + AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] + CoalescePartitionsExec + AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered + SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 ASC NULLS LAST] + MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +query ? +SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts + FROM sales_global AS s + GROUP BY s.group_key +---- +[50, 200] +[75, 80, 100] + +# test_ordering_sensitive_aggregation2 +query TT +EXPLAIN SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts + FROM sales_global AS s + GROUP BY s.group_key +---- +logical_plan +Projection: ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS FIRST) AS amounts + Aggregate: groupBy=[[s.group_key]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.group_key DESC NULLS FIRST]] + SubqueryAlias: s + TableScan: sales_global projection=[group_key, amount] +physical_plan +ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS FIRST)@1 as amounts] + AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] + CoalescePartitionsExec + AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered + SortExec: expr=[group_key@0 ASC NULLS LAST] + MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +query ? +SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts + FROM sales_global AS s + GROUP BY s.group_key +---- +[50, 200] +[75, 100, 80] + +# test_ordering_sensitive_aggregation3 +query TT +EXPLAIN SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.group_key +---- +logical_plan +Projection: ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.group_key]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] + SubqueryAlias: s + TableScan: sales_global projection=[group_key, amount] +physical_plan +ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] + CoalescePartitionsExec + AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered + SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 DESC] + MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +query ?I +SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.group_key +---- +[200, 50] 250 +[100, 80, 75] 255 + +# test_ordering_sensitive_aggregation4 +statement error DataFusion error: Error during planning: Conflicting ordering requirements in aggregate functions +SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2 + FROM sales_global AS s + GROUP BY s.group_key + From ecc5b0dab31949706013cefb36ce0127400f3d20 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 4 May 2023 18:29:42 +0300 Subject: [PATCH 04/24] update requirement --- .../core/src/physical_plan/aggregates/mod.rs | 17 +---------------- datafusion/core/tests/sql/window.rs | 5 +++-- .../sqllogictests/test_files/aggregate.slt | 13 ++++++------- 3 files changed, 10 insertions(+), 25 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 854f8b8e544e..46226d24a4b3 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -398,22 +398,7 @@ impl AggregateExec { )?; if let Some(req) = requirement { if group_by.groups.len() == 1 { - let requirement_prefix = output_group_expr_helper(&group_by); - let mut requirement = requirement_prefix - .into_iter() - .map(|expr| PhysicalSortRequirement::new(expr, None)) - .collect::>(); - let mut found = false; - for elem in &requirement { - if req.expr.eq(elem.expr()) { - found = true; - break; - } - } - if !found { - requirement.push(PhysicalSortRequirement::from(req)); - } - required_input_ordering = Some(requirement); + required_input_ordering = Some(vec![PhysicalSortRequirement::from(req)]); } else { return Err(DataFusionError::Plan( "Cannot run order sensitive aggregation in grouping set queries" diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 586561a2de9c..5e400580a228 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -525,6 +525,7 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_ordering_sensitive_aggregation() -> Result<()> { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::with_config(config); @@ -559,8 +560,8 @@ mod tests { let expected = { vec![ "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1]", - " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered", - " SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 DESC]", + " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)]", + " SortExec: expr=[amount@1 DESC]", ] }; diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 86267743edc6..3f328e5f33f9 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1676,8 +1676,8 @@ physical_plan ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered - SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 ASC NULLS LAST] + AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] + SortExec: expr=[amount@1 ASC NULLS LAST] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query ? @@ -1704,7 +1704,7 @@ ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered - SortExec: expr=[group_key@0 ASC NULLS LAST] + SortExec: expr=[group_key@0 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query ? @@ -1712,8 +1712,8 @@ SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts FROM sales_global AS s GROUP BY s.group_key ---- -[50, 200] [75, 100, 80] +[50, 200] # test_ordering_sensitive_aggregation3 query TT @@ -1731,8 +1731,8 @@ physical_plan ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered - SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 DESC] + AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] + SortExec: expr=[amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] query ?I @@ -1750,4 +1750,3 @@ SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2 FROM sales_global AS s GROUP BY s.group_key - From 7093fa7a14b710afe91e9cf2571050f24b0e79fc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 10:59:09 +0300 Subject: [PATCH 05/24] update tests --- .../core/src/physical_plan/aggregates/mod.rs | 29 +++- datafusion/core/tests/sql/window.rs | 15 +- .../sqllogictests/test_files/aggregate.slt | 145 ++++++++++-------- 3 files changed, 119 insertions(+), 70 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 46226d24a4b3..ad0538dfd307 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -390,7 +390,7 @@ impl AggregateExec { )?; let schema = Arc::new(schema); - let mut required_input_ordering = None; + let mut aggregator_requirement = None; if mode == AggregateMode::Partial || mode == AggregateMode::Single { let requirement = get_finest_requirement( &order_by_expr, @@ -398,7 +398,7 @@ impl AggregateExec { )?; if let Some(req) = requirement { if group_by.groups.len() == 1 { - required_input_ordering = Some(vec![PhysicalSortRequirement::from(req)]); + aggregator_requirement = Some(vec![PhysicalSortRequirement::from(req)]); } else { return Err(DataFusionError::Plan( "Cannot run order sensitive aggregation in grouping set queries" @@ -420,6 +420,31 @@ impl AggregateExec { let aggregation_ordering = calc_aggregation_ordering(&input, &group_by); + let mut required_input_ordering = None; + if let Some(AggregationOrdering{ordering, mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, ..}) = &aggregation_ordering{ + if let Some(aggregator_requirement) = aggregator_requirement { + let mut requirement = ordering + .iter() + .map(|sort_expr| PhysicalSortRequirement::from(sort_expr.clone())) + .collect::>(); + for req in aggregator_requirement { + let mut found = false; + for elem in &requirement { + if req.expr().eq(elem.expr()) { + found = true; + break; + } + } + if !found { + requirement.push(req); + } + } + required_input_ordering = Some(requirement); + } + } else { + required_input_ordering = aggregator_requirement; + } + Ok(AggregateExec { mode, group_by, diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 5e400580a228..ef84602350fc 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -546,11 +546,18 @@ mod tests { ) .await?; - let sql = "SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + let sql = "SELECT + ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 - FROM sales_global AS s + FROM (SELECT * + FROM sales_global + ORDER BY group_key) AS s GROUP BY s.group_key"; + // let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts + // FROM sales_global AS s + // GROUP BY s.group_key"; + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; @@ -560,8 +567,8 @@ mod tests { let expected = { vec![ "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1]", - " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)]", - " SortExec: expr=[amount@1 DESC]", + " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered", + " SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 DESC]", ] }; diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 3f328e5f33f9..816630a676b1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1646,107 +1646,124 @@ set datafusion.sql_parser.dialect = 'Generic'; # create a table for testing statement ok -CREATE TABLE sales_global (group_key INT, - sn INT PRIMARY KEY, +CREATE TABLE sales_global (zip_code INT, + country VARCHAR(3), + sn INT, ts TIMESTAMP, currency VARCHAR(3), amount INT ) as VALUES - (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), - (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + (0, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.00), + (0, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.00), + (1, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) # test_ordering_sensitive_aggregation +# ordering sensitive requirement should add a SortExec in the final plan. To satisfy amount ASC +# in the aggregation statement ok set datafusion.execution.target_partitions = 1; query TT -EXPLAIN SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts - FROM sales_global AS s - GROUP BY s.group_key +EXPLAIN SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts + FROM sales_global AS s + GROUP BY s.country ---- logical_plan -Projection: ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST) AS amounts - Aggregate: groupBy=[[s.group_key]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST]] +Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST) AS amounts + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST]] SubqueryAlias: s - TableScan: sales_global projection=[group_key, amount] + TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] - AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] + AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] + AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] SortExec: expr=[amount@1 ASC NULLS LAST] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -query ? -SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts - FROM sales_global AS s - GROUP BY s.group_key +query T? +SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts + FROM sales_global AS s + GROUP BY s.country ---- -[50, 200] -[75, 80, 100] +FRA [50, 200] +TUR [75, 100] +GRC [80] # test_ordering_sensitive_aggregation2 +# We should be able to satisfy the finest requirement among all aggregators, when we have multiple aggregators. +# Hence final plan should have SortExec: expr=[amount@1 DESC] to satisfy array_agg requirement. query TT -EXPLAIN SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts - FROM sales_global AS s - GROUP BY s.group_key +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.country ---- logical_plan -Projection: ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS FIRST) AS amounts - Aggregate: groupBy=[[s.group_key]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.group_key DESC NULLS FIRST]] +Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s - TableScan: sales_global projection=[group_key, amount] + TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.group_key DESC NULLS FIRST)@1 as amounts] - AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount)], ordering_mode=FullyOrdered - SortExec: expr=[group_key@0 DESC] + AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] + SortExec: expr=[amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -query ? -SELECT (ARRAY_AGG(s.amount ORDER BY s.group_key DESC)) AS amounts - FROM sales_global AS s - GROUP BY s.group_key +query T?I +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.country ---- -[75, 100, 80] -[50, 200] +FRA [200, 50] 250 +TUR [100, 75] 175 +GRC [80] 80 # test_ordering_sensitive_aggregation3 +# When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. +# test below should raise Plan Error. +statement error DataFusion error: Error during planning: Conflicting ordering requirements in aggregate functions +SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, + ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 + FROM sales_global AS s + GROUP BY s.country + +# test_ordering_sensitive_aggregation4 +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. query TT -EXPLAIN SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM sales_global AS s - GROUP BY s.group_key +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country ---- logical_plan -Projection: ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.group_key]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] +Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s - TableScan: sales_global projection=[group_key, amount] + Sort: sales_global.country ASC NULLS LAST + TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] - AggregateExec: mode=Final, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] - SortExec: expr=[amount@1 DESC] - MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] - -query ?I -SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM sales_global AS s - GROUP BY s.group_key +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered + SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] + MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +query T?I +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country ---- [200, 50] 250 [100, 80, 75] 255 - -# test_ordering_sensitive_aggregation4 -statement error DataFusion error: Error during planning: Conflicting ordering requirements in aggregate functions -SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2 - FROM sales_global AS s - GROUP BY s.group_key From e377ea943e3eccefe28972d52fc5431ecedaf8c5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 13:43:38 +0300 Subject: [PATCH 06/24] Add support for partiallyOrdered aggregation sensitive. --- .../physical_optimizer/sort_enforcement.rs | 13 ++- .../core/src/physical_plan/aggregates/mod.rs | 7 +- datafusion/core/tests/sql/window.rs | 44 +++++---- .../sqllogictests/test_files/aggregate.slt | 90 +++++++++++++++++-- 4 files changed, 130 insertions(+), 24 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 3e7ebf2525e2..671318da674d 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -49,7 +49,7 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{ BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, }; -use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; +use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan, displayable}; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::utils::{get_at_indices, longest_consecutive_prefix}; @@ -444,6 +444,13 @@ fn parallelize_sorts( })) } +fn print_plan(plan: &Arc) -> Result<()>{ + let formatted = displayable(plan.as_ref()).indent().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) +} + /// This function enforces sorting requirements and makes optimizations without /// violating these requirements whenever possible. fn ensure_sorting( @@ -454,6 +461,8 @@ fn ensure_sorting( return Ok(Transformed::No(requirements)); } let plan = requirements.plan; + println!("at the start"); + print_plan(&plan)?; let mut children = plan.children(); let mut sort_onwards = requirements.sort_onwards; if let Some(result) = analyze_immediate_sort_removal(&plan, &sort_onwards) { @@ -520,6 +529,8 @@ fn ensure_sorting( sort_onwards: vec![sort_onwards[0].clone()], })); } + print_plan(&plan)?; + println!("at the end"); Ok(Transformed::Yes(PlanWithCorrespondingSort { plan: plan.with_new_children(children)?, sort_onwards, diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index ad0538dfd307..4b06fa16fb3a 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -423,7 +423,8 @@ impl AggregateExec { let mut required_input_ordering = None; if let Some(AggregationOrdering{ordering, mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, ..}) = &aggregation_ordering{ if let Some(aggregator_requirement) = aggregator_requirement { - let mut requirement = ordering + let requirement_prefix = input.output_ordering().unwrap()[0..ordering.len()].to_vec(); + let mut requirement = requirement_prefix .iter() .map(|sort_expr| PhysicalSortRequirement::from(sort_expr.clone())) .collect::>(); @@ -444,6 +445,10 @@ impl AggregateExec { } else { required_input_ordering = aggregator_requirement; } + if mode == AggregateMode::Partial || mode == AggregateMode::Single { + println!("aggregation_ordering:{:?}", aggregation_ordering); + println!("required_input_ordering:{:?}", required_input_ordering); + } Ok(AggregateExec { mode, diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index ef84602350fc..52280145e35d 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -530,34 +530,48 @@ mod tests { let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::with_config(config); ctx.sql( - "CREATE TABLE sales_global ( - group_key INT, - sn INT PRIMARY KEY, + "CREATE TABLE sales_global (zip_code INT, + country VARCHAR(3), + sn INT, ts TIMESTAMP, currency VARCHAR(3), amount INT ) as VALUES - (0, 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 2, '2022-01-01 11:30:00'::timestamp, 'EUR', 75.00), - (0, 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 100.00), - (1, 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + (0, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.00), + (0, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.00), + (1, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) ", ) .await?; - let sql = "SELECT - ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY group_key) AS s - GROUP BY s.group_key"; + // let sql = "SELECT + // ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + // SUM(s.amount) AS sum1 + // FROM (SELECT * + // FROM sales_global + // ORDER BY group_key) AS s + // GROUP BY s.group_key"; // let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts // FROM sales_global AS s // GROUP BY s.group_key"; + let sql = "SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country, s.zip_code"; + + // let sql = "SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + // SUM(s.amount) AS sum1 + // FROM (SELECT * + // FROM sales_global + // ORDER BY country, zip_code) AS s + // GROUP BY zip_code, s.country"; + let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 816630a676b1..c5a8637cca51 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1653,11 +1653,12 @@ CREATE TABLE sales_global (zip_code INT, currency VARCHAR(3), amount INT ) as VALUES - (0, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.00), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.00), - (0, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.00), - (1, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) # test_ordering_sensitive_aggregation # ordering sensitive requirement should add a SortExec in the final plan. To satisfy amount ASC @@ -1688,9 +1689,9 @@ SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts FROM sales_global AS s GROUP BY s.country ---- +GRC [30, 80] FRA [50, 200] TUR [75, 100] -GRC [80] # test_ordering_sensitive_aggregation2 # We should be able to satisfy the finest requirement among all aggregators, when we have multiple aggregators. @@ -1722,7 +1723,7 @@ SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ---- FRA [200, 50] 250 TUR [100, 75] 175 -GRC [80] 80 +GRC [80, 30] 110 # test_ordering_sensitive_aggregation3 # When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. @@ -1737,6 +1738,7 @@ SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, # test_ordering_sensitive_aggregation4 # If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. +# This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. query TT EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -1765,5 +1767,79 @@ SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ORDER BY country) AS s GROUP BY s.country ---- -[200, 50] 250 -[100, 80, 75] 255 +FRA [200, 50] 250 +GRC [80, 30] 110 +TUR [100, 75] 175 + +# test_ordering_sensitive_aggregation5 +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. +# This test checks for whether we can satisfy aggregation requirement in PartiallyOrdered mode. +query TT +EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country, s.zip_code +---- +logical_plan +Projection: s.country, s.zip_code, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] + SubqueryAlias: s + 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, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@2 as amounts, SUM(s.amount)@3 as sum1] + AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered + SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] + MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +query TI?I +SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country, s.zip_code +---- +FRA 1 [200, 50] 250 +GRC 0 [80, 30] 110 +TUR 1 [100, 75] 175 + +# test_ordering_sensitive_aggregation6 +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# the existing ordering. When group by expressions contain aggregation requirement, we shouldn't append redundant expression. +# Hence in the final plan SortExec should be SortExec: expr=[country@0 DESC] not SortExec: expr=[country@0 ASC NULLS LAST,country@0 DESC] +query TT +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country +---- +logical_plan +Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.country DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST, SUM(s.amount)]] + SubqueryAlias: s + Sort: sales_global.country ASC NULLS LAST + TableScan: sales_global projection=[country, amount] +physical_plan +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.country DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered + SortPreservingMergeExec: [country@0 ASC NULLS LAST] + SortExec: expr=[country@0 ASC NULLS LAST] + MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] + +query T?I +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country +---- +FRA [200, 50] 250 +GRC [80, 30] 110 +TUR [100, 75] 175 From e0ce989c1995318f7128d01948af8e7e04435f6d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 14:14:59 +0300 Subject: [PATCH 07/24] Resolve linter errors --- .../physical_optimizer/dist_enforcement.rs | 1 + .../physical_optimizer/sort_enforcement.rs | 13 +-- .../core/src/physical_plan/aggregates/mod.rs | 13 ++- datafusion/core/tests/sql/window.rs | 84 ------------------- .../substrait/src/logical_plan/producer.rs | 2 +- 5 files changed, 13 insertions(+), 100 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index 6abaae065a24..23c2fa0e13e8 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -369,6 +369,7 @@ where } } +#[allow(clippy::too_many_arguments)] fn reorder_aggregate_keys( agg_plan: Arc, parent_required: &[Arc], diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 671318da674d..3e7ebf2525e2 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -49,7 +49,7 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{ BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, }; -use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan, displayable}; +use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::utils::{get_at_indices, longest_consecutive_prefix}; @@ -444,13 +444,6 @@ fn parallelize_sorts( })) } -fn print_plan(plan: &Arc) -> Result<()>{ - let formatted = displayable(plan.as_ref()).indent().to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) -} - /// This function enforces sorting requirements and makes optimizations without /// violating these requirements whenever possible. fn ensure_sorting( @@ -461,8 +454,6 @@ fn ensure_sorting( return Ok(Transformed::No(requirements)); } let plan = requirements.plan; - println!("at the start"); - print_plan(&plan)?; let mut children = plan.children(); let mut sort_onwards = requirements.sort_onwards; if let Some(result) = analyze_immediate_sort_removal(&plan, &sort_onwards) { @@ -529,8 +520,6 @@ fn ensure_sorting( sort_onwards: vec![sort_onwards[0].clone()], })); } - print_plan(&plan)?; - println!("at the end"); Ok(Transformed::Yes(PlanWithCorrespondingSort { plan: plan.with_new_children(children)?, sort_onwards, diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 4b06fa16fb3a..b75adc405d51 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -398,7 +398,8 @@ impl AggregateExec { )?; if let Some(req) = requirement { if group_by.groups.len() == 1 { - aggregator_requirement = Some(vec![PhysicalSortRequirement::from(req)]); + aggregator_requirement = + Some(vec![PhysicalSortRequirement::from(req)]); } else { return Err(DataFusionError::Plan( "Cannot run order sensitive aggregation in grouping set queries" @@ -421,9 +422,15 @@ impl AggregateExec { let aggregation_ordering = calc_aggregation_ordering(&input, &group_by); let mut required_input_ordering = None; - if let Some(AggregationOrdering{ordering, mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, ..}) = &aggregation_ordering{ + if let Some(AggregationOrdering { + ordering, + mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, + .. + }) = &aggregation_ordering + { if let Some(aggregator_requirement) = aggregator_requirement { - let requirement_prefix = input.output_ordering().unwrap()[0..ordering.len()].to_vec(); + let requirement_prefix = + input.output_ordering().unwrap()[0..ordering.len()].to_vec(); let mut requirement = requirement_prefix .iter() .map(|sort_expr| PhysicalSortRequirement::from(sort_expr.clone())) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 52280145e35d..7e7ed031f1ca 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -62,7 +62,6 @@ async fn window_frame_creation_type_checking() -> Result<()> { mod tests { use super::*; - use arrow::util::pretty::print_batches; use datafusion::test_util::{get_test_context, get_test_context2}; #[tokio::test] @@ -523,87 +522,4 @@ mod tests { assert_batches_eq!(expected, &actual); Ok(()) } - - #[tokio::test] - #[ignore] - async fn test_ordering_sensitive_aggregation() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - ctx.sql( - "CREATE TABLE sales_global (zip_code INT, - country VARCHAR(3), - sn INT, - ts TIMESTAMP, - currency VARCHAR(3), - amount INT - ) as VALUES - (0, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.00), - (0, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.00), - (1, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) - ", - ) - .await?; - - // let sql = "SELECT - // ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - // SUM(s.amount) AS sum1 - // FROM (SELECT * - // FROM sales_global - // ORDER BY group_key) AS s - // GROUP BY s.group_key"; - - // let sql = "SELECT (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts - // FROM sales_global AS s - // GROUP BY s.group_key"; - - let sql = "SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country, s.zip_code"; - - // let sql = "SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - // SUM(s.amount) AS sum1 - // FROM (SELECT * - // FROM sales_global - // ORDER BY country, zip_code) AS s - // GROUP BY zip_code, s.country"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - let formatted = displayable(physical_plan.as_ref()).indent().to_string(); - // We should produce `BoundedWindowAggExec`s that only works in Sorted mode, since source is finite. - // To satisfy, requirements for Sorted mode. We should introduce `SortExec`s through physical plan. - let expected = { - vec![ - "ProjectionExec: expr=[ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1]", - " AggregateExec: mode=Single, gby=[group_key@0 as group_key], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered", - " SortExec: expr=[group_key@0 ASC NULLS LAST,amount@1 DESC]", - ] - }; - - let actual: Vec<&str> = formatted.trim().lines().collect(); - let actual_len = actual.len(); - let actual_trim_last = &actual[..actual_len - 1]; - assert_eq!( - expected, actual_trim_last, - "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - let actual = execute_to_batches(&ctx, sql).await; - - let expected = vec![ - "+---------------+------+", - "| amounts | sum1 |", - "+---------------+------+", - "| [200, 50] | 250 |", - "| [100, 80, 75] | 255 |", - "+---------------+------+", - ]; - assert_batches_eq!(expected, &actual); - Ok(()) - } } diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 679b01b4f0eb..30d42ad55823 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -415,7 +415,7 @@ pub fn to_substrait_agg_measure( ), ) -> Result { match expr { - Expr::AggregateFunction(expr::AggregateFunction { fun, args, distinct, filter, order_by }) => { + Expr::AggregateFunction(expr::AggregateFunction { fun, args, distinct, filter, order_by: _order_by }) => { let mut arguments: Vec = vec![]; for arg in args { arguments.push(FunctionArgument { arg_type: Some(ArgType::Value(to_substrait_rex(arg, schema, extension_info)?)) }); From 550416eb5911c30f7658a2d74e2fd01b6a3f6c29 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 15:14:58 +0300 Subject: [PATCH 08/24] update comments --- .../core/src/physical_plan/aggregates/mod.rs | 33 +++++++++++-------- datafusion/core/src/physical_plan/planner.rs | 9 +++-- .../sqllogictests/test_files/aggregate.slt | 20 +++++------ datafusion/expr/src/expr.rs | 2 +- 4 files changed, 36 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 51d740649ff5..023c6b7523a6 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -342,6 +342,8 @@ fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec], eq_properties: &[EquivalentClass], @@ -389,6 +391,9 @@ impl AggregateExec { let schema = Arc::new(schema); let mut aggregator_requirement = None; + // Ordering sensitive requirement makes sense only in Partial and Single mode. + // Because in other modes, all groups are collapsed. Hence their input schema may not contain + // expressions in the requirement. if mode == AggregateMode::Partial || mode == AggregateMode::Single { let requirement = get_finest_requirement( &order_by_expr, @@ -396,15 +401,7 @@ impl AggregateExec { input.ordering_equivalence_properties().classes(), )?; if let Some(req) = requirement { - if group_by.groups.len() == 1 { - aggregator_requirement = - Some(vec![PhysicalSortRequirement::from(req)]); - } else { - return Err(DataFusionError::Plan( - "Cannot run order sensitive aggregation in grouping set queries" - .to_string(), - )); - } + aggregator_requirement = Some(vec![PhysicalSortRequirement::from(req)]); } } @@ -423,13 +420,25 @@ impl AggregateExec { let mut required_input_ordering = None; if let Some(AggregationOrdering { ordering, + // If the mode of the AggregateExec is FullyOrdered or PartiallyOrdered + // (e.g AggregateExec runs with bounded memory without breaking pipeline) + // We append aggregator ordering requirement to the existing ordering that enables + // to run executor with bounded memory. + // In this way, we can still run executor in the same mode, + // and can satisfy required ordering for the aggregators. mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, .. }) = &aggregation_ordering { if let Some(aggregator_requirement) = aggregator_requirement { + // Get the section of the input ordering that enables us to run executor in the + // GroupByOrderMode::FullyOrdered or GroupByOrderMode::PartiallyOrdered. let requirement_prefix = - input.output_ordering().unwrap()[0..ordering.len()].to_vec(); + if let Some(existing_ordering) = input.output_ordering() { + existing_ordering[0..ordering.len()].to_vec() + } else { + vec![] + }; let mut requirement = requirement_prefix .iter() .map(|sort_expr| PhysicalSortRequirement::from(sort_expr.clone())) @@ -451,10 +460,6 @@ impl AggregateExec { } else { required_input_ordering = aggregator_requirement; } - if mode == AggregateMode::Partial || mode == AggregateMode::Single { - println!("aggregation_ordering:{:?}", aggregation_ordering); - println!("required_input_ordering:{:?}", required_input_ordering); - } Ok(AggregateExec { mode, diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index e0787491c40a..cfb84225b7b2 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -701,6 +701,7 @@ impl DefaultPhysicalPlanner { logical_input_schema, &physical_input_schema, session_state)?; + let agg_filter = aggr_expr .iter() .map(|e| { @@ -1611,9 +1612,11 @@ pub fn create_window_expr( ) } -type AggregateExprWithOptionalFilter = ( +type AggregateExprWithOptionalArgs = ( Arc, + // Keeps Filter clause if any Option>, + // Keeps ordering requirement if any Option, ); @@ -1624,7 +1627,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( logical_input_schema: &DFSchema, physical_input_schema: &Schema, execution_props: &ExecutionProps, -) -> Result { +) -> Result { match e { Expr::AggregateFunction(AggregateFunction { fun, @@ -1724,7 +1727,7 @@ pub fn create_aggregate_expr_and_maybe_filter( logical_input_schema: &DFSchema, physical_input_schema: &Schema, execution_props: &ExecutionProps, -) -> Result { +) -> Result { // unpack (nested) aliased logical expressions, e.g. "sum(col) as total" let (name, e) = match e { Expr::Alias(sub_expr, alias) => (alias.clone(), sub_expr.as_ref()), diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 34296c540f34..ff83ae4e93f7 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1743,12 +1743,12 @@ EXPLAIN SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST) AS amounts +Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.amount ASC NULLS LAST) AS amounts Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST]] SubqueryAlias: s TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] @@ -1774,12 +1774,12 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] @@ -1819,13 +1819,13 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s Sort: sales_global.country ASC NULLS LAST TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -1855,13 +1855,13 @@ EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) GROUP BY s.country, s.zip_code ---- logical_plan -Projection: s.country, s.zip_code, ARRAYAGG(s.amount) FILTER (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, s.zip_code, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s 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, ARRAYAGG(s.amount) FILTER (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, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST)@2 as amounts, SUM(s.amount)@3 as sum1] AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -1891,13 +1891,13 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) FILTER (ORDER BY s.country DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.country DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s Sort: sales_global.country ASC NULLS LAST TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) FILTER (ORDER BY s.country DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.country DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered SortPreservingMergeExec: [country@0 ASC NULLS LAST] SortExec: expr=[country@0 ASC NULLS LAST] diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 46ed3553a67c..7d3f649cdab4 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1330,7 +1330,7 @@ fn create_name(e: &Expr) -> Result { name = format!("{name} FILTER (WHERE {fe})"); }; if let Some(order_by) = order_by { - name = format!("{name} FILTER (ORDER BY {order_by})"); + name = format!("{name} (ORDER BY {order_by})"); }; Ok(name) } From d72cc8f974aed3d433e9bf516c0eb94543c796c1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 15:30:24 +0300 Subject: [PATCH 09/24] minor changes --- .../sqllogictests/test_files/aggregate.slt | 20 +++++++++---------- datafusion/expr/src/expr.rs | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index ff83ae4e93f7..cc29beb1e6d1 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1743,12 +1743,12 @@ EXPLAIN SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.amount ASC NULLS LAST) AS amounts +Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST AS amounts Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST]] SubqueryAlias: s TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.amount ASC NULLS LAST)@1 as amounts] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST@1 as amounts] AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] @@ -1774,12 +1774,12 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] @@ -1819,13 +1819,13 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s Sort: sales_global.country ASC NULLS LAST TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -1855,13 +1855,13 @@ EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) GROUP BY s.country, s.zip_code ---- logical_plan -Projection: s.country, s.zip_code, ARRAYAGG(s.amount) (ORDER BY s.amount DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, s.zip_code, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s 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, ARRAYAGG(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, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST@2 as amounts, SUM(s.amount)@3 as sum1] AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -1891,13 +1891,13 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) (ORDER BY s.country DESC NULLS FIRST) AS amounts, SUM(s.amount) AS sum1 +Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST, SUM(s.amount)]] SubqueryAlias: s Sort: sales_global.country ASC NULLS LAST TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) (ORDER BY s.country DESC NULLS FIRST)@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered SortPreservingMergeExec: [country@0 ASC NULLS LAST] SortExec: expr=[country@0 ASC NULLS LAST] diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 7d3f649cdab4..df91c149efff 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1330,7 +1330,7 @@ fn create_name(e: &Expr) -> Result { name = format!("{name} FILTER (WHERE {fe})"); }; if let Some(order_by) = order_by { - name = format!("{name} (ORDER BY {order_by})"); + name = format!("{name} ORDER BY {order_by:?}"); }; Ok(name) } From 3dcf0797229a2d973086f1eda1aede74267928fd Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 16:41:14 +0300 Subject: [PATCH 10/24] retract changes in generated --- datafusion/proto/src/generated/prost.rs | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index ef83eba2bba2..fd3cdc129235 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -692,8 +692,6 @@ pub struct AggregateExprNode { pub distinct: bool, #[prost(message, optional, boxed, tag = "4")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "5")] - pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -704,8 +702,6 @@ pub struct AggregateUdfExprNode { pub args: ::prost::alloc::vec::Vec, #[prost(message, optional, boxed, tag = "3")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "4")] - pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1824,12 +1820,6 @@ pub struct MaybeFilter { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct MaybePhysicalSortExpr { - #[prost(message, optional, tag = "1")] - pub expr: ::core::option::Option, -} -#[allow(clippy::derive_partial_eq_without_eq)] -#[derive(Clone, PartialEq, ::prost::Message)] pub struct AggregateExecNode { #[prost(message, repeated, tag = "1")] pub group_expr: ::prost::alloc::vec::Vec, @@ -1852,8 +1842,6 @@ pub struct AggregateExecNode { pub groups: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "10")] pub filter_expr: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "11")] - pub order_by_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] From a817887fce7e9a4d3f67fabce3a882c1b79e3dce Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 5 May 2023 17:01:22 +0300 Subject: [PATCH 11/24] update proto files --- datafusion/proto/proto/datafusion.proto | 7 ++ datafusion/proto/src/generated/pbjson.rs | 145 +++++++++++++++++++++++ datafusion/proto/src/generated/prost.rs | 12 ++ 3 files changed, 164 insertions(+) diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 7d02fda86c33..716db153aa1a 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -568,12 +568,14 @@ message AggregateExprNode { repeated LogicalExprNode expr = 2; bool distinct = 3; LogicalExprNode filter = 4; + LogicalExprNode order_by = 5; } message AggregateUDFExprNode { string fun_name = 1; repeated LogicalExprNode args = 2; LogicalExprNode filter = 3; + LogicalExprNode order_by = 4; } message ScalarUDFExprNode { @@ -1276,6 +1278,10 @@ message MaybeFilter { PhysicalExprNode expr = 1; } +message MaybePhysicalSortExpr { + PhysicalSortExprNode expr = 1; +} + message AggregateExecNode { repeated PhysicalExprNode group_expr = 1; repeated PhysicalExprNode aggr_expr = 2; @@ -1288,6 +1294,7 @@ message AggregateExecNode { repeated PhysicalExprNode null_expr = 8; repeated bool groups = 9; repeated MaybeFilter filter_expr = 10; + repeated MaybePhysicalSortExpr order_by_expr = 11; } message GlobalLimitExecNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 553f3f291146..2e09266fc2a1 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -36,6 +36,9 @@ impl serde::Serialize for AggregateExecNode { if !self.filter_expr.is_empty() { len += 1; } + if !self.order_by_expr.is_empty() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateExecNode", len)?; if !self.group_expr.is_empty() { struct_ser.serialize_field("groupExpr", &self.group_expr)?; @@ -69,6 +72,9 @@ impl serde::Serialize for AggregateExecNode { if !self.filter_expr.is_empty() { struct_ser.serialize_field("filterExpr", &self.filter_expr)?; } + if !self.order_by_expr.is_empty() { + struct_ser.serialize_field("orderByExpr", &self.order_by_expr)?; + } struct_ser.end() } } @@ -96,6 +102,8 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { "groups", "filter_expr", "filterExpr", + "order_by_expr", + "orderByExpr", ]; #[allow(clippy::enum_variant_names)] @@ -110,6 +118,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { NullExpr, Groups, FilterExpr, + OrderByExpr, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -141,6 +150,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { "nullExpr" | "null_expr" => Ok(GeneratedField::NullExpr), "groups" => Ok(GeneratedField::Groups), "filterExpr" | "filter_expr" => Ok(GeneratedField::FilterExpr), + "orderByExpr" | "order_by_expr" => Ok(GeneratedField::OrderByExpr), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -170,6 +180,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { let mut null_expr__ = None; let mut groups__ = None; let mut filter_expr__ = None; + let mut order_by_expr__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::GroupExpr => { @@ -232,6 +243,12 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { } filter_expr__ = Some(map.next_value()?); } + GeneratedField::OrderByExpr => { + if order_by_expr__.is_some() { + return Err(serde::de::Error::duplicate_field("orderByExpr")); + } + order_by_expr__ = Some(map.next_value()?); + } } } Ok(AggregateExecNode { @@ -245,6 +262,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { null_expr: null_expr__.unwrap_or_default(), groups: groups__.unwrap_or_default(), filter_expr: filter_expr__.unwrap_or_default(), + order_by_expr: order_by_expr__.unwrap_or_default(), }) } } @@ -271,6 +289,9 @@ impl serde::Serialize for AggregateExprNode { if self.filter.is_some() { len += 1; } + if self.order_by.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateExprNode", len)?; if self.aggr_function != 0 { let v = AggregateFunction::from_i32(self.aggr_function) @@ -286,6 +307,9 @@ impl serde::Serialize for AggregateExprNode { if let Some(v) = self.filter.as_ref() { struct_ser.serialize_field("filter", v)?; } + if let Some(v) = self.order_by.as_ref() { + struct_ser.serialize_field("orderBy", v)?; + } struct_ser.end() } } @@ -301,6 +325,8 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { "expr", "distinct", "filter", + "order_by", + "orderBy", ]; #[allow(clippy::enum_variant_names)] @@ -309,6 +335,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { Expr, Distinct, Filter, + OrderBy, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -334,6 +361,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { "expr" => Ok(GeneratedField::Expr), "distinct" => Ok(GeneratedField::Distinct), "filter" => Ok(GeneratedField::Filter), + "orderBy" | "order_by" => Ok(GeneratedField::OrderBy), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -357,6 +385,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { let mut expr__ = None; let mut distinct__ = None; let mut filter__ = None; + let mut order_by__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::AggrFunction => { @@ -383,6 +412,12 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { } filter__ = map.next_value()?; } + GeneratedField::OrderBy => { + if order_by__.is_some() { + return Err(serde::de::Error::duplicate_field("orderBy")); + } + order_by__ = map.next_value()?; + } } } Ok(AggregateExprNode { @@ -390,6 +425,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { expr: expr__.unwrap_or_default(), distinct: distinct__.unwrap_or_default(), filter: filter__, + order_by: order_by__, }) } } @@ -743,6 +779,9 @@ impl serde::Serialize for AggregateUdfExprNode { if self.filter.is_some() { len += 1; } + if self.order_by.is_some() { + len += 1; + } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateUDFExprNode", len)?; if !self.fun_name.is_empty() { struct_ser.serialize_field("funName", &self.fun_name)?; @@ -753,6 +792,9 @@ impl serde::Serialize for AggregateUdfExprNode { if let Some(v) = self.filter.as_ref() { struct_ser.serialize_field("filter", v)?; } + if let Some(v) = self.order_by.as_ref() { + struct_ser.serialize_field("orderBy", v)?; + } struct_ser.end() } } @@ -767,6 +809,8 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { "funName", "args", "filter", + "order_by", + "orderBy", ]; #[allow(clippy::enum_variant_names)] @@ -774,6 +818,7 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { FunName, Args, Filter, + OrderBy, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -798,6 +843,7 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { "funName" | "fun_name" => Ok(GeneratedField::FunName), "args" => Ok(GeneratedField::Args), "filter" => Ok(GeneratedField::Filter), + "orderBy" | "order_by" => Ok(GeneratedField::OrderBy), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -820,6 +866,7 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { let mut fun_name__ = None; let mut args__ = None; let mut filter__ = None; + let mut order_by__ = None; while let Some(k) = map.next_key()? { match k { GeneratedField::FunName => { @@ -840,12 +887,19 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { } filter__ = map.next_value()?; } + GeneratedField::OrderBy => { + if order_by__.is_some() { + return Err(serde::de::Error::duplicate_field("orderBy")); + } + order_by__ = map.next_value()?; + } } } Ok(AggregateUdfExprNode { fun_name: fun_name__.unwrap_or_default(), args: args__.unwrap_or_default(), filter: filter__, + order_by: order_by__, }) } } @@ -11484,6 +11538,97 @@ impl<'de> serde::Deserialize<'de> for MaybeFilter { deserializer.deserialize_struct("datafusion.MaybeFilter", FIELDS, GeneratedVisitor) } } +impl serde::Serialize for MaybePhysicalSortExpr { + #[allow(deprecated)] + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + use serde::ser::SerializeStruct; + let mut len = 0; + if self.expr.is_some() { + len += 1; + } + let mut struct_ser = serializer.serialize_struct("datafusion.MaybePhysicalSortExpr", len)?; + if let Some(v) = self.expr.as_ref() { + struct_ser.serialize_field("expr", v)?; + } + struct_ser.end() + } +} +impl<'de> serde::Deserialize<'de> for MaybePhysicalSortExpr { + #[allow(deprecated)] + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + const FIELDS: &[&str] = &[ + "expr", + ]; + + #[allow(clippy::enum_variant_names)] + enum GeneratedField { + Expr, + } + impl<'de> serde::Deserialize<'de> for GeneratedField { + fn deserialize(deserializer: D) -> std::result::Result + where + D: serde::Deserializer<'de>, + { + struct GeneratedVisitor; + + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = GeneratedField; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!(formatter, "expected one of: {:?}", &FIELDS) + } + + #[allow(unused_variables)] + fn visit_str(self, value: &str) -> std::result::Result + where + E: serde::de::Error, + { + match value { + "expr" => Ok(GeneratedField::Expr), + _ => Err(serde::de::Error::unknown_field(value, FIELDS)), + } + } + } + deserializer.deserialize_identifier(GeneratedVisitor) + } + } + struct GeneratedVisitor; + impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { + type Value = MaybePhysicalSortExpr; + + fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + formatter.write_str("struct datafusion.MaybePhysicalSortExpr") + } + + fn visit_map(self, mut map: V) -> std::result::Result + where + V: serde::de::MapAccess<'de>, + { + let mut expr__ = None; + while let Some(k) = map.next_key()? { + match k { + GeneratedField::Expr => { + if expr__.is_some() { + return Err(serde::de::Error::duplicate_field("expr")); + } + expr__ = map.next_value()?; + } + } + } + Ok(MaybePhysicalSortExpr { + expr: expr__, + }) + } + } + deserializer.deserialize_struct("datafusion.MaybePhysicalSortExpr", FIELDS, GeneratedVisitor) + } +} impl serde::Serialize for NegativeNode { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index fd3cdc129235..ef83eba2bba2 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -692,6 +692,8 @@ pub struct AggregateExprNode { pub distinct: bool, #[prost(message, optional, boxed, tag = "4")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, boxed, tag = "5")] + pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -702,6 +704,8 @@ pub struct AggregateUdfExprNode { pub args: ::prost::alloc::vec::Vec, #[prost(message, optional, boxed, tag = "3")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, optional, boxed, tag = "4")] + pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1820,6 +1824,12 @@ pub struct MaybeFilter { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] +pub struct MaybePhysicalSortExpr { + #[prost(message, optional, tag = "1")] + pub expr: ::core::option::Option, +} +#[allow(clippy::derive_partial_eq_without_eq)] +#[derive(Clone, PartialEq, ::prost::Message)] pub struct AggregateExecNode { #[prost(message, repeated, tag = "1")] pub group_expr: ::prost::alloc::vec::Vec, @@ -1842,6 +1852,8 @@ pub struct AggregateExecNode { pub groups: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "10")] pub filter_expr: ::prost::alloc::vec::Vec, + #[prost(message, repeated, tag = "11")] + pub order_by_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] From 740a6057c869f8463d14f5ed93aeb9a6ffa989ae Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 5 May 2023 19:31:04 -0500 Subject: [PATCH 12/24] Simplifications --- .../core/src/physical_plan/aggregates/mod.rs | 67 +++++++++---------- datafusion/core/src/physical_plan/planner.rs | 59 ++++++++-------- datafusion/expr/src/expr.rs | 10 ++- .../proto/src/physical_plan/from_proto.rs | 4 +- 4 files changed, 63 insertions(+), 77 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 023c6b7523a6..4aa3043932bb 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -41,7 +41,7 @@ use datafusion_physical_expr::{ equivalence::project_equivalence_properties, expressions::{Avg, CastExpr, Column, Sum}, normalize_out_expr_with_columns_map, - utils::{convert_to_expr, get_indices_of_matching_exprs}, + utils::{convert_to_expr, get_indices_of_matching_exprs, normalize_sort_expr}, AggregateExpr, EquivalentClass, OrderingEquivalentClass, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; @@ -56,7 +56,6 @@ mod utils; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; -use datafusion_physical_expr::utils::normalize_sort_expr; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -342,32 +341,37 @@ fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec], eq_properties: &[EquivalentClass], ordering_eq_properties: &[OrderingEquivalentClass], ) -> Result> { - let mut res: Option = None; - for elem in order_by_expr.iter().flatten() { - if let Some(res) = &res { - let res_normalized = - normalize_sort_expr(res.clone(), eq_properties, ordering_eq_properties); - let elem_normalized = - normalize_sort_expr(elem.clone(), eq_properties, ordering_eq_properties); - if !res_normalized.eq(&elem_normalized) { - println!("Conflicting requirements"); + let mut result: Option = None; + let mut normalized_result: Option = None; + for item in order_by_expr.iter().flatten() { + if let Some(normalized_expr) = &normalized_result { + let normalized_item = + normalize_sort_expr(item.clone(), eq_properties, ordering_eq_properties); + if normalized_item.ne(normalized_expr) { return Err(DataFusionError::Plan( "Conflicting ordering requirements in aggregate functions" .to_string(), )); } } else { - res = Some(elem.clone()) + result = Some(item.clone()); + normalized_result = Some(normalize_sort_expr( + item.clone(), + eq_properties, + ordering_eq_properties, + )); } } - Ok(res) + Ok(result) } impl AggregateExec { @@ -391,17 +395,17 @@ impl AggregateExec { let schema = Arc::new(schema); let mut aggregator_requirement = None; - // Ordering sensitive requirement makes sense only in Partial and Single mode. - // Because in other modes, all groups are collapsed. Hence their input schema may not contain - // expressions in the requirement. + // Ordering requirement makes sense only in Partial and Single modes. + // In other modes, all groups are collapsed, therefore their input schema + // can not contain expressions in the requirement. if mode == AggregateMode::Partial || mode == AggregateMode::Single { let requirement = get_finest_requirement( &order_by_expr, input.equivalence_properties().classes(), input.ordering_equivalence_properties().classes(), )?; - if let Some(req) = requirement { - aggregator_requirement = Some(vec![PhysicalSortRequirement::from(req)]); + if let Some(expr) = requirement { + aggregator_requirement = Some(vec![PhysicalSortRequirement::from(expr)]); } } @@ -420,19 +424,17 @@ impl AggregateExec { let mut required_input_ordering = None; if let Some(AggregationOrdering { ordering, - // If the mode of the AggregateExec is FullyOrdered or PartiallyOrdered - // (e.g AggregateExec runs with bounded memory without breaking pipeline) - // We append aggregator ordering requirement to the existing ordering that enables - // to run executor with bounded memory. - // In this way, we can still run executor in the same mode, - // and can satisfy required ordering for the aggregators. + // If the mode is FullyOrdered or PartiallyOrdered (i.e. we are + // running with bounded memory, without breaking pipeline), then + // we append aggregator ordering requirement to the existing + // ordering. This way, we can still run with bounded memory. mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, .. }) = &aggregation_ordering { if let Some(aggregator_requirement) = aggregator_requirement { - // Get the section of the input ordering that enables us to run executor in the - // GroupByOrderMode::FullyOrdered or GroupByOrderMode::PartiallyOrdered. + // Get the section of the input ordering that enables us to run in the + // FullyOrdered or PartiallyOrdered mode: let requirement_prefix = if let Some(existing_ordering) = input.output_ordering() { existing_ordering[0..ordering.len()].to_vec() @@ -444,14 +446,7 @@ impl AggregateExec { .map(|sort_expr| PhysicalSortRequirement::from(sort_expr.clone())) .collect::>(); for req in aggregator_requirement { - let mut found = false; - for elem in &requirement { - if req.expr().eq(elem.expr()) { - found = true; - break; - } - } - if !found { + if requirement.iter().all(|item| req.expr().ne(item.expr())) { requirement.push(req); } } diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index cfb84225b7b2..46b117b476ad 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -74,7 +74,7 @@ use datafusion_physical_expr::expressions::Literal; use datafusion_sql::utils::window_expr_common_partition_keys; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; -use itertools::Itertools; +use itertools::{multiunzip, Itertools}; use log::{debug, trace}; use std::collections::HashMap; use std::fmt::Write; @@ -713,12 +713,8 @@ impl DefaultPhysicalPlanner { ) }) .collect::>>()?; - let (mut aggregates,mut filters, mut order_bys) = (vec![], vec![], vec![]); - for (aggregate, filter, order_by) in agg_filter.into_iter(){ - aggregates.push(aggregate); - filters.push(filter); - order_bys.push(order_by); - } + + let (aggregates, filters, order_bys) : (Vec<_>, Vec<_>, Vec<_>) = multiunzip(agg_filter.into_iter()); let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, @@ -1614,9 +1610,9 @@ pub fn create_window_expr( type AggregateExprWithOptionalArgs = ( Arc, - // Keeps Filter clause if any + // The filter clause, if any Option>, - // Keeps ordering requirement if any + // Ordering requirement, if any Option, ); @@ -1662,7 +1658,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( &args, physical_input_schema, name, - ); + )?; let order_by = match order_by { Some(e) => Some(create_physical_sort_expr( e, @@ -1672,7 +1668,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( )?), None => None, }; - Ok((agg_expr?, filter, order_by)) + Ok((agg_expr, filter, order_by)) } Expr::AggregateUDF { fun, @@ -1750,29 +1746,28 @@ pub fn create_physical_sort_expr( input_schema: &Schema, execution_props: &ExecutionProps, ) -> Result { - match e { - Expr::Sort(expr::Sort { - expr, - asc, - nulls_first, - }) => { - let options = SortOptions { + if let Expr::Sort(expr::Sort { + expr, + asc, + nulls_first, + }) = e + { + Ok(PhysicalSortExpr { + expr: create_physical_expr( + expr, + input_dfschema, + input_schema, + execution_props, + )?, + options: SortOptions { descending: !asc, nulls_first: *nulls_first, - }; - Ok(PhysicalSortExpr { - expr: create_physical_expr( - expr, - input_dfschema, - input_schema, - execution_props, - )?, - options, - }) - } - _ => Err(DataFusionError::Plan( - "Sort only accepts sort expressions".to_string(), - )), + }, + }) + } else { + Err(DataFusionError::Internal( + "Expects a sort expression".to_string(), + )) } } diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index df91c149efff..8b2fe6214f9d 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -28,9 +28,7 @@ use crate::AggregateUDF; use crate::Operator; use crate::ScalarUDF; use arrow::datatypes::DataType; -use datafusion_common::Result; -use datafusion_common::{plan_err, Column}; -use datafusion_common::{DataFusionError, ScalarValue}; +use datafusion_common::{plan_err, Column, DataFusionError, Result, ScalarValue}; use std::collections::HashSet; use std::fmt; use std::fmt::{Display, Formatter, Write}; @@ -177,7 +175,7 @@ pub enum Expr { args: Vec, /// Optional filter applied prior to aggregating filter: Option>, - /// Optional order by applied prior to aggregating + /// Optional ORDER BY applied prior to aggregating order_by: Option>, }, /// Returns whether the list contains the expr value. @@ -436,7 +434,7 @@ pub struct AggregateFunction { pub distinct: bool, /// Optional filter pub filter: Option>, - /// Optional Ordering + /// Optional ordering pub order_by: Option>, } @@ -1344,7 +1342,7 @@ fn create_name(e: &Expr) -> Result { for e in args { names.push(create_name(e)?); } - let mut info = "".to_string(); + let mut info = String::new(); if let Some(fe) = filter { info += &format!(" FILTER (WHERE {fe})"); } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index fb7ab7328552..1290db075259 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -82,9 +82,7 @@ pub fn parse_physical_sort_expr( }; Ok(PhysicalSortExpr { expr, options }) } else { - Err(DataFusionError::Execution( - "Cannot parse proto to PhysicalSortExpr".to_string(), - )) + Err(proto_error("Unexpected empty physical expression")) } } From 8209c46a04042976a4dd8440f2dbfae7538e73c9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 8 May 2023 11:00:32 +0300 Subject: [PATCH 13/24] Make types consistent in schema, and data --- .../sqllogictests/test_files/aggregate.slt | 52 +++++++++---------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index b92ba9d5efcc..8c0e798c0d02 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1738,14 +1738,14 @@ CREATE TABLE sales_global (zip_code INT, sn INT, ts TIMESTAMP, currency VARCHAR(3), - amount INT + amount FLOAT ) as VALUES - (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.00), - (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.00), - (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.00), - (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.00), - (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.00), - (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.00) + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), + (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) # test_ordering_sensitive_aggregation # ordering sensitive requirement should add a SortExec in the final plan. To satisfy amount ASC @@ -1776,9 +1776,9 @@ SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts FROM sales_global AS s GROUP BY s.country ---- -GRC [30, 80] -FRA [50, 200] -TUR [75, 100] +GRC [30.0, 80.0] +FRA [50.0, 200.0] +TUR [75.0, 100.0] # test_ordering_sensitive_aggregation2 # We should be able to satisfy the finest requirement among all aggregators, when we have multiple aggregators. @@ -1802,15 +1802,15 @@ ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount SortExec: expr=[amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -query T?I +query T?R SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM sales_global AS s GROUP BY s.country ---- -FRA [200, 50] 250 -TUR [100, 75] 175 -GRC [80, 30] 110 +FRA [200.0, 50.0] 250 +TUR [100.0, 75.0] 175 +GRC [80.0, 30.0] 110 # test_ordering_sensitive_aggregation3 # When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. @@ -1846,7 +1846,7 @@ ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -query T?I +query T?R SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -1854,9 +1854,9 @@ SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ORDER BY country) AS s GROUP BY s.country ---- -FRA [200, 50] 250 -GRC [80, 30] 110 -TUR [100, 75] 175 +FRA [200.0, 50.0] 250 +GRC [80.0, 30.0] 110 +TUR [100.0, 75.0] 175 # test_ordering_sensitive_aggregation5 # If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to @@ -1882,7 +1882,7 @@ ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAYAGG(s.a SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -query TI?I +query TI?R SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -1890,9 +1890,9 @@ SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amou ORDER BY country) AS s GROUP BY s.country, s.zip_code ---- -FRA 1 [200, 50] 250 -GRC 0 [80, 30] 110 -TUR 1 [100, 75] 175 +FRA 1 [200.0, 50.0] 250 +GRC 0 [80.0, 30.0] 110 +TUR 1 [100.0, 75.0] 175 # test_ordering_sensitive_aggregation6 # If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to @@ -1919,7 +1919,7 @@ ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.countr SortExec: expr=[country@0 ASC NULLS LAST] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] -query T?I +query T?R SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -1927,6 +1927,6 @@ SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ORDER BY country) AS s GROUP BY s.country ---- -FRA [200, 50] 250 -GRC [80, 30] 110 -TUR [100, 75] 175 +FRA [200.0, 50.0] 250 +GRC [80.0, 30.0] 110 +TUR [100.0, 75.0] 175 From d93b187336839b3d822c320c4ae89c6d5a1bf453 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 8 May 2023 17:20:15 +0300 Subject: [PATCH 14/24] Update todos --- datafusion/core/src/physical_plan/planner.rs | 1 + datafusion/substrait/src/logical_plan/producer.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index 46b117b476ad..e88d42b1cc2b 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -205,6 +205,7 @@ fn create_physical_name(e: &Expr, is_first_expr: bool) -> Result { filter, order_by, } => { + // TODO: Add support for filter and order by in AggregateUDF if filter.is_some() { return Err(DataFusionError::Execution( "aggregate expression with filter is not supported".to_string(), diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 30d42ad55823..0523c51cc312 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -415,6 +415,7 @@ pub fn to_substrait_agg_measure( ), ) -> Result { match expr { + // TODO: Once substrait supports order by, add handling for it. Expr::AggregateFunction(expr::AggregateFunction { fun, args, distinct, filter, order_by: _order_by }) => { let mut arguments: Vec = vec![]; for arg in args { From 5bb8dde9afcea42c1e0ff18102cbb55491620d84 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 8 May 2023 18:36:08 +0300 Subject: [PATCH 15/24] Convert API to vector --- .../physical_optimizer/dist_enforcement.rs | 2 +- .../core/src/physical_plan/aggregates/mod.rs | 19 +++--- datafusion/core/src/physical_plan/planner.rs | 40 +++++++----- .../sqllogictests/test_files/aggregate.slt | 30 ++++----- datafusion/expr/src/expr.rs | 10 +-- datafusion/expr/src/tree_node/expr.rs | 10 ++- datafusion/proto/proto/datafusion.proto | 10 +-- datafusion/proto/src/generated/pbjson.rs | 61 ++++++++++--------- datafusion/proto/src/generated/prost.rs | 16 ++--- .../proto/src/logical_plan/from_proto.rs | 19 +++++- datafusion/proto/src/logical_plan/to_proto.rs | 14 +++-- datafusion/proto/src/physical_plan/mod.rs | 20 ++++-- .../proto/src/physical_plan/to_proto.rs | 15 +++-- datafusion/sql/src/expr/mod.rs | 5 +- .../substrait/src/logical_plan/consumer.rs | 2 +- 15 files changed, 164 insertions(+), 109 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index fb0e56f4813f..4c30170ace3d 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -376,7 +376,7 @@ fn reorder_aggregate_keys( group_by: &PhysicalGroupBy, aggr_expr: &[Arc], filter_expr: &[Option>], - order_by_expr: &[Option], + order_by_expr: &[Option>], agg_input: Arc, input_schema: &SchemaRef, ) -> Result { diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 4aa3043932bb..ace5582ff237 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -230,7 +230,7 @@ pub struct AggregateExec { /// FILTER (WHERE clause) expression for each aggregate expression pub(crate) filter_expr: Vec>>, /// (ORDER BY clause) expression for each aggregate expression - pub(crate) order_by_expr: Vec>, + pub(crate) order_by_expr: Vec>>, /// Input plan, could be a partial aggregate or the input to the aggregate pub(crate) input: Arc, /// Schema after the aggregate is applied @@ -346,7 +346,7 @@ fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec], + order_by_expr: &[Option>], eq_properties: &[EquivalentClass], ordering_eq_properties: &[OrderingEquivalentClass], ) -> Result> { @@ -354,8 +354,11 @@ fn get_finest_requirement( let mut normalized_result: Option = None; for item in order_by_expr.iter().flatten() { if let Some(normalized_expr) = &normalized_result { - let normalized_item = - normalize_sort_expr(item.clone(), eq_properties, ordering_eq_properties); + let normalized_item = normalize_sort_expr( + item[0].clone(), + eq_properties, + ordering_eq_properties, + ); if normalized_item.ne(normalized_expr) { return Err(DataFusionError::Plan( "Conflicting ordering requirements in aggregate functions" @@ -363,9 +366,9 @@ fn get_finest_requirement( )); } } else { - result = Some(item.clone()); + result = Some(item[0].clone()); normalized_result = Some(normalize_sort_expr( - item.clone(), + item[0].clone(), eq_properties, ordering_eq_properties, )); @@ -381,7 +384,7 @@ impl AggregateExec { group_by: PhysicalGroupBy, aggr_expr: Vec>, filter_expr: Vec>>, - order_by_expr: Vec>, + order_by_expr: Vec>>, input: Arc, input_schema: SchemaRef, ) -> Result { @@ -498,7 +501,7 @@ impl AggregateExec { } /// ORDER BY clause expression for each aggregate expression - pub fn order_by_expr(&self) -> &[Option] { + pub fn order_by_expr(&self) -> &[Option>] { &self.order_by_expr } diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index e88d42b1cc2b..77b4ed9d953f 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -1613,8 +1613,8 @@ type AggregateExprWithOptionalArgs = ( Arc, // The filter clause, if any Option>, - // Ordering requirement, if any - Option, + // Ordering requirements, if any + Option>, ); /// Create an aggregate expression with a name from a logical expression @@ -1661,12 +1661,18 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( name, )?; let order_by = match order_by { - Some(e) => Some(create_physical_sort_expr( - e, - logical_input_schema, - physical_input_schema, - execution_props, - )?), + Some(e) => Some( + e.iter() + .map(|expr| { + create_physical_sort_expr( + expr, + logical_input_schema, + physical_input_schema, + execution_props, + ) + }) + .collect::>>()?, + ), None => None, }; Ok((agg_expr, filter, order_by)) @@ -1699,12 +1705,18 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( None => None, }; let order_by = match order_by { - Some(e) => Some(create_physical_sort_expr( - e, - logical_input_schema, - physical_input_schema, - execution_props, - )?), + Some(e) => Some( + e.iter() + .map(|expr| { + create_physical_sort_expr( + expr, + logical_input_schema, + physical_input_schema, + execution_props, + ) + }) + .collect::>>()?, + ), None => None, }; diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 8c0e798c0d02..28b9cb332ad7 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1759,12 +1759,12 @@ EXPLAIN SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST AS amounts - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST]] +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount ASC NULLS LAST] AS amounts + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]]] SubqueryAlias: s TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount ASC NULLS LAST@1 as amounts] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]@1 as amounts] AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] @@ -1790,12 +1790,12 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] SubqueryAlias: s TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] CoalescePartitionsExec AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] @@ -1835,13 +1835,13 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] SubqueryAlias: s Sort: sales_global.country ASC NULLS LAST TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -1871,13 +1871,13 @@ EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) GROUP BY s.country, s.zip_code ---- logical_plan -Projection: s.country, s.zip_code, ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.amount DESC NULLS FIRST, SUM(s.amount)]] +Projection: s.country, s.zip_code, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] SubqueryAlias: s 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, ARRAYAGG(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, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] @@ -1907,13 +1907,13 @@ EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts GROUP BY s.country ---- logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST, SUM(s.amount)]] +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(s.amount)]] SubqueryAlias: s Sort: sales_global.country ASC NULLS LAST TableScan: sales_global projection=[country, amount] physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY s.country DESC NULLS FIRST@1 as amounts, SUM(s.amount)@2 as sum1] +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered SortPreservingMergeExec: [country@0 ASC NULLS LAST] SortExec: expr=[country@0 ASC NULLS LAST] diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 8b2fe6214f9d..041e5977b46f 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -176,7 +176,7 @@ pub enum Expr { /// Optional filter applied prior to aggregating filter: Option>, /// Optional ORDER BY applied prior to aggregating - order_by: Option>, + order_by: Option>, }, /// Returns whether the list contains the expr value. InList { @@ -435,7 +435,7 @@ pub struct AggregateFunction { /// Optional filter pub filter: Option>, /// Optional ordering - pub order_by: Option>, + pub order_by: Option>, } impl AggregateFunction { @@ -444,7 +444,7 @@ impl AggregateFunction { args: Vec, distinct: bool, filter: Option>, - order_by: Option>, + order_by: Option>, ) -> Self { Self { fun, @@ -971,7 +971,7 @@ impl fmt::Debug for Expr { write!(f, " FILTER (WHERE {fe})")?; } if let Some(ob) = order_by { - write!(f, " ORDER BY {ob}")?; + write!(f, " ORDER BY {:?}", ob)?; } Ok(()) } @@ -1347,7 +1347,7 @@ fn create_name(e: &Expr) -> Result { info += &format!(" FILTER (WHERE {fe})"); } if let Some(ob) = order_by { - info += &format!(" ORDER BY ({ob})"); + info += &format!(" ORDER BY ({:?})", ob); } Ok(format!("{}({}){}", fun.name, names.join(","), info)) } diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index 924a2999740d..12f9054aa6ce 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -104,7 +104,7 @@ impl TreeNode for Expr { expr_vec.push(f.as_ref().clone()); } if let Some(o) = order_by { - expr_vec.push(o.as_ref().clone()); + expr_vec.extend(o.clone()); } expr_vec @@ -329,7 +329,13 @@ impl TreeNode for Expr { args: transform_vec(args, &mut transform)?, fun, filter: transform_option_box(filter, &mut transform)?, - order_by: transform_option_box(order_by, &mut transform)?, + order_by: { + if let Some(order_by) = order_by { + Some(transform_vec(order_by, &mut transform)?) + } else { + None + } + }, }, Expr::InList { expr, diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 716db153aa1a..3ed7361b0d29 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -568,14 +568,14 @@ message AggregateExprNode { repeated LogicalExprNode expr = 2; bool distinct = 3; LogicalExprNode filter = 4; - LogicalExprNode order_by = 5; + repeated LogicalExprNode order_by = 5; } message AggregateUDFExprNode { string fun_name = 1; repeated LogicalExprNode args = 2; LogicalExprNode filter = 3; - LogicalExprNode order_by = 4; + repeated LogicalExprNode order_by = 4; } message ScalarUDFExprNode { @@ -1278,8 +1278,8 @@ message MaybeFilter { PhysicalExprNode expr = 1; } -message MaybePhysicalSortExpr { - PhysicalSortExprNode expr = 1; +message MaybePhysicalSortExprs { + repeated PhysicalSortExprNode sort_expr = 1; } message AggregateExecNode { @@ -1294,7 +1294,7 @@ message AggregateExecNode { repeated PhysicalExprNode null_expr = 8; repeated bool groups = 9; repeated MaybeFilter filter_expr = 10; - repeated MaybePhysicalSortExpr order_by_expr = 11; + repeated MaybePhysicalSortExprs order_by_expr = 11; } message GlobalLimitExecNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 2e09266fc2a1..5baec89725b6 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -289,7 +289,7 @@ impl serde::Serialize for AggregateExprNode { if self.filter.is_some() { len += 1; } - if self.order_by.is_some() { + if !self.order_by.is_empty() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateExprNode", len)?; @@ -307,8 +307,8 @@ impl serde::Serialize for AggregateExprNode { if let Some(v) = self.filter.as_ref() { struct_ser.serialize_field("filter", v)?; } - if let Some(v) = self.order_by.as_ref() { - struct_ser.serialize_field("orderBy", v)?; + if !self.order_by.is_empty() { + struct_ser.serialize_field("orderBy", &self.order_by)?; } struct_ser.end() } @@ -416,7 +416,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { if order_by__.is_some() { return Err(serde::de::Error::duplicate_field("orderBy")); } - order_by__ = map.next_value()?; + order_by__ = Some(map.next_value()?); } } } @@ -425,7 +425,7 @@ impl<'de> serde::Deserialize<'de> for AggregateExprNode { expr: expr__.unwrap_or_default(), distinct: distinct__.unwrap_or_default(), filter: filter__, - order_by: order_by__, + order_by: order_by__.unwrap_or_default(), }) } } @@ -779,7 +779,7 @@ impl serde::Serialize for AggregateUdfExprNode { if self.filter.is_some() { len += 1; } - if self.order_by.is_some() { + if !self.order_by.is_empty() { len += 1; } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateUDFExprNode", len)?; @@ -792,8 +792,8 @@ impl serde::Serialize for AggregateUdfExprNode { if let Some(v) = self.filter.as_ref() { struct_ser.serialize_field("filter", v)?; } - if let Some(v) = self.order_by.as_ref() { - struct_ser.serialize_field("orderBy", v)?; + if !self.order_by.is_empty() { + struct_ser.serialize_field("orderBy", &self.order_by)?; } struct_ser.end() } @@ -891,7 +891,7 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { if order_by__.is_some() { return Err(serde::de::Error::duplicate_field("orderBy")); } - order_by__ = map.next_value()?; + order_by__ = Some(map.next_value()?); } } } @@ -899,7 +899,7 @@ impl<'de> serde::Deserialize<'de> for AggregateUdfExprNode { fun_name: fun_name__.unwrap_or_default(), args: args__.unwrap_or_default(), filter: filter__, - order_by: order_by__, + order_by: order_by__.unwrap_or_default(), }) } } @@ -11538,7 +11538,7 @@ impl<'de> serde::Deserialize<'de> for MaybeFilter { deserializer.deserialize_struct("datafusion.MaybeFilter", FIELDS, GeneratedVisitor) } } -impl serde::Serialize for MaybePhysicalSortExpr { +impl serde::Serialize for MaybePhysicalSortExprs { #[allow(deprecated)] fn serialize(&self, serializer: S) -> std::result::Result where @@ -11546,29 +11546,30 @@ impl serde::Serialize for MaybePhysicalSortExpr { { use serde::ser::SerializeStruct; let mut len = 0; - if self.expr.is_some() { + if !self.sort_expr.is_empty() { len += 1; } - let mut struct_ser = serializer.serialize_struct("datafusion.MaybePhysicalSortExpr", len)?; - if let Some(v) = self.expr.as_ref() { - struct_ser.serialize_field("expr", v)?; + let mut struct_ser = serializer.serialize_struct("datafusion.MaybePhysicalSortExprs", len)?; + if !self.sort_expr.is_empty() { + struct_ser.serialize_field("sortExpr", &self.sort_expr)?; } struct_ser.end() } } -impl<'de> serde::Deserialize<'de> for MaybePhysicalSortExpr { +impl<'de> serde::Deserialize<'de> for MaybePhysicalSortExprs { #[allow(deprecated)] fn deserialize(deserializer: D) -> std::result::Result where D: serde::Deserializer<'de>, { const FIELDS: &[&str] = &[ - "expr", + "sort_expr", + "sortExpr", ]; #[allow(clippy::enum_variant_names)] enum GeneratedField { - Expr, + SortExpr, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -11590,7 +11591,7 @@ impl<'de> serde::Deserialize<'de> for MaybePhysicalSortExpr { E: serde::de::Error, { match value { - "expr" => Ok(GeneratedField::Expr), + "sortExpr" | "sort_expr" => Ok(GeneratedField::SortExpr), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -11600,33 +11601,33 @@ impl<'de> serde::Deserialize<'de> for MaybePhysicalSortExpr { } struct GeneratedVisitor; impl<'de> serde::de::Visitor<'de> for GeneratedVisitor { - type Value = MaybePhysicalSortExpr; + type Value = MaybePhysicalSortExprs; fn expecting(&self, formatter: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - formatter.write_str("struct datafusion.MaybePhysicalSortExpr") + formatter.write_str("struct datafusion.MaybePhysicalSortExprs") } - fn visit_map(self, mut map: V) -> std::result::Result + fn visit_map(self, mut map: V) -> std::result::Result where V: serde::de::MapAccess<'de>, { - let mut expr__ = None; + let mut sort_expr__ = None; while let Some(k) = map.next_key()? { match k { - GeneratedField::Expr => { - if expr__.is_some() { - return Err(serde::de::Error::duplicate_field("expr")); + GeneratedField::SortExpr => { + if sort_expr__.is_some() { + return Err(serde::de::Error::duplicate_field("sortExpr")); } - expr__ = map.next_value()?; + sort_expr__ = Some(map.next_value()?); } } } - Ok(MaybePhysicalSortExpr { - expr: expr__, + Ok(MaybePhysicalSortExprs { + sort_expr: sort_expr__.unwrap_or_default(), }) } } - deserializer.deserialize_struct("datafusion.MaybePhysicalSortExpr", FIELDS, GeneratedVisitor) + deserializer.deserialize_struct("datafusion.MaybePhysicalSortExprs", FIELDS, GeneratedVisitor) } } impl serde::Serialize for NegativeNode { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index ef83eba2bba2..6dafdf3e0c58 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -692,8 +692,8 @@ pub struct AggregateExprNode { pub distinct: bool, #[prost(message, optional, boxed, tag = "4")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "5")] - pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, repeated, tag = "5")] + pub order_by: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -704,8 +704,8 @@ pub struct AggregateUdfExprNode { pub args: ::prost::alloc::vec::Vec, #[prost(message, optional, boxed, tag = "3")] pub filter: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(message, optional, boxed, tag = "4")] - pub order_by: ::core::option::Option<::prost::alloc::boxed::Box>, + #[prost(message, repeated, tag = "4")] + pub order_by: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1824,9 +1824,9 @@ pub struct MaybeFilter { } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] -pub struct MaybePhysicalSortExpr { - #[prost(message, optional, tag = "1")] - pub expr: ::core::option::Option, +pub struct MaybePhysicalSortExprs { + #[prost(message, repeated, tag = "1")] + pub sort_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] @@ -1853,7 +1853,7 @@ pub struct AggregateExecNode { #[prost(message, repeated, tag = "10")] pub filter_expr: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "11")] - pub order_by_expr: ::prost::alloc::vec::Vec, + pub order_by_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 7d8fb5b40088..8d1bcdb8ac40 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -981,7 +981,7 @@ pub fn parse_expr( .collect::, _>>()?, expr.distinct, parse_optional_expr(expr.filter.as_deref(), registry)?.map(Box::new), - parse_optional_expr(expr.order_by.as_deref(), registry)?.map(Box::new), + parse_vec_expr(&expr.order_by, registry)?, ))) } ExprType::Alias(alias) => Ok(Expr::Alias( @@ -1374,8 +1374,7 @@ pub fn parse_expr( .collect::, Error>>()?, filter: parse_optional_expr(pb.filter.as_deref(), registry)? .map(Box::new), - order_by: parse_optional_expr(pb.order_by.as_deref(), registry)? - .map(Box::new), + order_by: parse_vec_expr(&pb.order_by, registry)?, }) } @@ -1468,6 +1467,20 @@ pub fn from_proto_binary_op(op: &str) -> Result { } } +fn parse_vec_expr( + p: &[protobuf::LogicalExprNode], + registry: &dyn FunctionRegistry, +) -> Result>, Error> { + let res = p + .iter() + .map(|elem| { + parse_expr(elem, registry).map_err(|e| DataFusionError::Plan(e.to_string())) + }) + .collect::>>()?; + // Convert empty vector to None. + Ok((!res.is_empty()).then_some(res)) +} + fn parse_optional_expr( p: Option<&protobuf::LogicalExprNode>, registry: &dyn FunctionRegistry, diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index c695dd5af2ef..847b933b981c 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -670,8 +670,11 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { None => None, }, order_by: match order_by { - Some(e) => Some(Box::new(e.as_ref().try_into()?)), - None => None, + Some(e) => e + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + None => vec![], }, }; Self { @@ -727,8 +730,11 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { None => None, }, order_by: match order_by { - Some(e) => Some(Box::new(e.as_ref().try_into()?)), - None => None, + Some(e) => e + .iter() + .map(|expr| expr.try_into()) + .collect::, _>>()?, + None => vec![], }, }, ))), diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index b473b0a52f75..040417c38a19 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -420,12 +420,22 @@ impl AsExecutionPlan for PhysicalPlanNode { .order_by_expr .iter() .map(|expr| { - let x = expr.expr.as_ref().map(|e| { - parse_physical_sort_expr(e, registry, &physical_schema) - }); - x.transpose() + let x = expr + .sort_expr + .iter() + .map(|e| { + parse_physical_sort_expr(e, registry, &physical_schema) + }) + .collect::>>(); + match x { + Ok(exprs) => { + // Convert empty vec to None. + Ok((!exprs.is_empty()).then_some(exprs)) + } + Err(e) => Err(e), + } }) - .collect::, _>>()?; + .collect::>>()?; let physical_aggr_expr: Vec> = hash_agg .aggr_expr diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index b30c9c2598c5..60357834a1c9 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -531,14 +531,17 @@ impl TryFrom>> for protobuf::MaybeFilter { } } -impl TryFrom> for protobuf::MaybePhysicalSortExpr { +impl TryFrom>> for protobuf::MaybePhysicalSortExprs { type Error = DataFusionError; - fn try_from(sort_expr: Option) -> Result { - match sort_expr { - None => Ok(protobuf::MaybePhysicalSortExpr { expr: None }), - Some(sort_expr) => Ok(protobuf::MaybePhysicalSortExpr { - expr: Some(sort_expr.try_into()?), + fn try_from(sort_exprs: Option>) -> Result { + match sort_exprs { + None => Ok(protobuf::MaybePhysicalSortExprs { sort_expr: vec![] }), + Some(sort_exprs) => Ok(protobuf::MaybePhysicalSortExprs { + sort_expr: sort_exprs + .into_iter() + .map(|sort_expr| sort_expr.try_into()) + .collect::>>()?, }), } } diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 75e2e9301b1d..cc3f3140c8cd 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -318,11 +318,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { } = array_agg; let order_by = if let Some(order_by) = order_by { - Some(Box::new(self.order_by_to_sort_expr( + // TODO: Once sqlparser supports multiple order by clause, handle it + Some(vec![self.order_by_to_sort_expr( *order_by, input_schema, planner_context, - )?)) + )?]) } else { None }; diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index 31654e863789..24bac58dd523 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -551,7 +551,7 @@ pub async fn from_substrait_agg_func( input_schema: &DFSchema, extensions: &HashMap, filter: Option>, - order_by: Option>, + order_by: Option>, distinct: bool, ) -> Result> { let mut args: Vec = vec![]; From 72985f8b8dec87bc5ab125c7a1876c3025f85b63 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 9 May 2023 12:41:17 +0300 Subject: [PATCH 16/24] Convert get_finest to handle Vector inputs --- .../core/src/physical_plan/aggregates/mod.rs | 151 +++++++++++++----- datafusion/physical-expr/src/utils.rs | 2 +- 2 files changed, 115 insertions(+), 38 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index ace5582ff237..9e42f0e8fb37 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -41,9 +41,9 @@ use datafusion_physical_expr::{ equivalence::project_equivalence_properties, expressions::{Avg, CastExpr, Column, Sum}, normalize_out_expr_with_columns_map, - utils::{convert_to_expr, get_indices_of_matching_exprs, normalize_sort_expr}, - AggregateExpr, EquivalentClass, OrderingEquivalentClass, PhysicalExpr, - PhysicalSortExpr, PhysicalSortRequirement, + utils::{convert_to_expr, get_indices_of_matching_exprs}, + AggregateExpr, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, }; use std::any::Any; use std::collections::HashMap; @@ -56,6 +56,7 @@ mod utils; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; +use datafusion_physical_expr::utils::ordering_satisfy_concrete; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -343,35 +344,43 @@ fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec EquivalenceProperties, + F2: Fn() -> OrderingEquivalenceProperties, +>( order_by_expr: &[Option>], - eq_properties: &[EquivalentClass], - ordering_eq_properties: &[OrderingEquivalentClass], -) -> Result> { - let mut result: Option = None; - let mut normalized_result: Option = None; - for item in order_by_expr.iter().flatten() { - if let Some(normalized_expr) = &normalized_result { - let normalized_item = normalize_sort_expr( - item[0].clone(), - eq_properties, - ordering_eq_properties, - ); - if normalized_item.ne(normalized_expr) { - return Err(DataFusionError::Plan( - "Conflicting ordering requirements in aggregate functions" - .to_string(), - )); + eq_properties: F, + ordering_eq_properties: F2, +) -> Result>> { + let mut result: Option> = None; + for fn_reqs in order_by_expr.iter().flatten() { + if let Some(result) = &mut result { + if ordering_satisfy_concrete( + result, + fn_reqs, + &eq_properties, + &ordering_eq_properties, + ) { + // do not update result, result already satisfies the requirement for current function + continue; } - } else { - result = Some(item[0].clone()); - normalized_result = Some(normalize_sort_expr( - item[0].clone(), - eq_properties, - ordering_eq_properties, + if ordering_satisfy_concrete( + fn_reqs, + result, + &eq_properties, + &ordering_eq_properties, + ) { + // update result with fn_reqs, fn_reqs satisfy the existing requirement and itself. + *result = fn_reqs.clone(); + continue; + } + // If either of the requirements satisfy other this means that, requirement are conflicting. + return Err(DataFusionError::Plan( + "Conflicting ordering requirements in aggregate functions".to_string(), )); + } else { + result = Some(fn_reqs.clone()); } } Ok(result) @@ -404,12 +413,15 @@ impl AggregateExec { if mode == AggregateMode::Partial || mode == AggregateMode::Single { let requirement = get_finest_requirement( &order_by_expr, - input.equivalence_properties().classes(), - input.ordering_equivalence_properties().classes(), + || input.equivalence_properties(), + || input.ordering_equivalence_properties(), )?; - if let Some(expr) = requirement { - aggregator_requirement = Some(vec![PhysicalSortRequirement::from(expr)]); - } + aggregator_requirement = requirement.map(|exprs| { + exprs + .into_iter() + .map(PhysicalSortRequirement::from) + .collect::>() + }); } // construct a map from the input columns to the output columns of the Aggregation @@ -1053,7 +1065,8 @@ mod tests { use crate::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use crate::from_slice::FromSlice; use crate::physical_plan::aggregates::{ - get_working_mode, AggregateExec, AggregateMode, PhysicalGroupBy, + get_finest_requirement, get_working_mode, AggregateExec, AggregateMode, + PhysicalGroupBy, }; use crate::physical_plan::expressions::{col, Avg}; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; @@ -1064,8 +1077,13 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::{DataFusionError, Result, ScalarValue}; - use datafusion_physical_expr::expressions::{lit, ApproxDistinct, Count, Median}; - use datafusion_physical_expr::{AggregateExpr, PhysicalExpr, PhysicalSortExpr}; + use datafusion_physical_expr::expressions::{ + lit, ApproxDistinct, Column, Count, Median, + }; + use datafusion_physical_expr::{ + AggregateExpr, EquivalenceProperties, OrderedColumn, + OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + }; use futures::{FutureExt, Stream}; use std::any::Any; use std::sync::Arc; @@ -1699,4 +1717,63 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_get_finest_requirements() -> Result<()> { + let test_schema = create_test_schema()?; + // Assume column a and b are aliases + // Assume also that a ASC and c DESC describe the same global ordering for the table. (Since they are ordering equivalent). + let options1 = SortOptions { + descending: false, + nulls_first: false, + }; + let options2 = SortOptions { + descending: true, + nulls_first: true, + }; + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + let col_a = Column::new("a", 0); + let col_b = Column::new("b", 1); + let col_c = Column::new("c", 2); + let col_d = Column::new("d", 3); + eq_properties.add_equal_conditions((&col_a, &col_b)); + let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); + ordering_eq_properties.add_equal_conditions(( + &OrderedColumn::new(col_a.clone(), options1), + &OrderedColumn::new(col_c.clone(), options2), + )); + + let order_by_exprs = vec![ + None, + Some(vec![PhysicalSortExpr { + expr: Arc::new(col_a.clone()), + options: options1, + }]), + Some(vec![PhysicalSortExpr { + expr: Arc::new(col_b.clone()), + options: options1, + }]), + Some(vec![PhysicalSortExpr { + expr: Arc::new(col_c), + options: options2, + }]), + Some(vec![ + PhysicalSortExpr { + expr: Arc::new(col_a), + options: options1, + }, + PhysicalSortExpr { + expr: Arc::new(col_d), + options: options1, + }, + ]), + ]; + let res = get_finest_requirement( + &order_by_exprs, + || eq_properties.clone(), + || ordering_eq_properties.clone(), + )?; + assert_eq!(res, order_by_exprs[4]); + Ok(()) + } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 3b186ddf8ab2..c3cfb67ffe5a 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -322,7 +322,7 @@ pub fn ordering_satisfy< /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the /// provided [`PhysicalSortExpr`]s. -fn ordering_satisfy_concrete< +pub fn ordering_satisfy_concrete< F: FnOnce() -> EquivalenceProperties, F2: FnOnce() -> OrderingEquivalenceProperties, >( From 4ae5c256e3c35ee8cc4746a8e13389eb889ed59d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 9 May 2023 12:49:02 +0300 Subject: [PATCH 17/24] simplifications, update comment --- datafusion/core/src/physical_plan/aggregates/mod.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 9e42f0e8fb37..fb9c23b12d64 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -375,7 +375,8 @@ fn get_finest_requirement< *result = fn_reqs.clone(); continue; } - // If either of the requirements satisfy other this means that, requirement are conflicting. + // If either of the requirements satisfy the other, this means that requirements are conflicting. + // Currently we do not have support for this functionality. return Err(DataFusionError::Plan( "Conflicting ordering requirements in aggregate functions".to_string(), )); @@ -457,8 +458,8 @@ impl AggregateExec { vec![] }; let mut requirement = requirement_prefix - .iter() - .map(|sort_expr| PhysicalSortRequirement::from(sort_expr.clone())) + .into_iter() + .map(PhysicalSortRequirement::from) .collect::>(); for req in aggregator_requirement { if requirement.iter().all(|item| req.expr().ne(item.expr())) { From c59ce9bb96f4e6fc14ff55de925ab8d9c21cc603 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 10 May 2023 20:59:23 -0500 Subject: [PATCH 18/24] Minor code simplifications --- .../core/src/physical_plan/aggregates/mod.rs | 14 +++++++------ datafusion/proto/src/physical_plan/mod.rs | 20 ++++++------------- 2 files changed, 14 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index fb9c23b12d64..11ff7a18fe8d 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -41,7 +41,7 @@ use datafusion_physical_expr::{ equivalence::project_equivalence_properties, expressions::{Avg, CastExpr, Column, Sum}, normalize_out_expr_with_columns_map, - utils::{convert_to_expr, get_indices_of_matching_exprs}, + utils::{convert_to_expr, get_indices_of_matching_exprs, ordering_satisfy_concrete}, AggregateExpr, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; @@ -56,7 +56,6 @@ mod utils; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; -use datafusion_physical_expr::utils::ordering_satisfy_concrete; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -362,7 +361,8 @@ fn get_finest_requirement< &eq_properties, &ordering_eq_properties, ) { - // do not update result, result already satisfies the requirement for current function + // Do not update the result as it already satisfies current + // function's requirement: continue; } if ordering_satisfy_concrete( @@ -371,12 +371,14 @@ fn get_finest_requirement< &eq_properties, &ordering_eq_properties, ) { - // update result with fn_reqs, fn_reqs satisfy the existing requirement and itself. + // Update result with current function's requirements, as it is + // a finer requirement than what we currently have. *result = fn_reqs.clone(); continue; } - // If either of the requirements satisfy the other, this means that requirements are conflicting. - // Currently we do not have support for this functionality. + // If either of the requirements satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. return Err(DataFusionError::Plan( "Conflicting ordering requirements in aggregate functions".to_string(), )); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 040417c38a19..94e26f2cf2b4 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -409,31 +409,23 @@ impl AsExecutionPlan for PhysicalPlanNode { .filter_expr .iter() .map(|expr| { - let x = expr - .expr + expr.expr .as_ref() - .map(|e| parse_physical_expr(e, registry, &physical_schema)); - x.transpose() + .map(|e| parse_physical_expr(e, registry, &physical_schema)) + .transpose() }) .collect::, _>>()?; let physical_order_by_expr = hash_agg .order_by_expr .iter() .map(|expr| { - let x = expr - .sort_expr + expr.sort_expr .iter() .map(|e| { parse_physical_sort_expr(e, registry, &physical_schema) }) - .collect::>>(); - match x { - Ok(exprs) => { - // Convert empty vec to None. - Ok((!exprs.is_empty()).then_some(exprs)) - } - Err(e) => Err(e), - } + .collect::>>() + .map(|exprs| (!exprs.is_empty()).then_some(exprs)) }) .collect::>>()?; From 90aa195283e7bd186fe9bbcbd299fcd2a634e79c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 11 May 2023 10:18:52 +0300 Subject: [PATCH 19/24] Update comment --- datafusion/core/src/physical_plan/aggregates/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 11ff7a18fe8d..7d1f445bb846 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -376,7 +376,7 @@ fn get_finest_requirement< *result = fn_reqs.clone(); continue; } - // If either of the requirements satisfy the other, this means + // If neither of the requirements satisfy the other, this means // requirements are conflicting. Currently, we do not support // conflicting requirements. return Err(DataFusionError::Plan( From 9abfe65fc445a60c14fc8b6fb91cf5c74cd6f6e2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 11 May 2023 15:57:08 +0300 Subject: [PATCH 20/24] Update documents --- docs/source/user-guide/sql/aggregate_functions.md | 4 ++-- docs/source/user-guide/sql/select.md | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index 497cf78aadeb..06bc91643dc4 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -124,10 +124,10 @@ sum(expression) ### `array_agg` -Returns an array created from the expression elements. +Returns an array created from the expression elements. If ordering requirement is given, elements are inserted in the order of required ordering. ``` -array_agg(expression) +array_agg(expression [ORDER BY expression]) ``` #### Arguments diff --git a/docs/source/user-guide/sql/select.md b/docs/source/user-guide/sql/select.md index 68be88d7cff3..cb73e0852030 100644 --- a/docs/source/user-guide/sql/select.md +++ b/docs/source/user-guide/sql/select.md @@ -189,6 +189,15 @@ Example: SELECT a, b, MAX(c) FROM table GROUP BY a, b ``` +Some aggregation functions accept optional ordering requirement, such as `ARRAY_AGG`. If a requirement is given, +aggregation is calculated in the order of the requirement. + +Example: + +```sql +SELECT a, b, ARRAY_AGG(c, ORDER BY d) FROM table GROUP BY a, b +``` + ## HAVING clause Example: From 344e184da2c30cedb2f29f8dc18daf409aeeed54 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 12 May 2023 12:01:18 +0300 Subject: [PATCH 21/24] fix projection push down failure bug --- .../sqllogictests/test_files/aggregate.slt | 200 ------------------ .../sqllogictests/test_files/groupby.slt | 198 +++++++++++++++++ datafusion/expr/src/expr.rs | 4 + datafusion/expr/src/tree_node/expr.rs | 19 +- 4 files changed, 219 insertions(+), 202 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index cfcde5e91ca0..0c0a2c49b493 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -1742,203 +1742,3 @@ select max(x_dict) from value_dict group by x_dict % 2 order by max(x_dict); ---- 4 5 - -# create a table for testing -statement ok -CREATE TABLE sales_global (zip_code INT, - country VARCHAR(3), - sn INT, - ts TIMESTAMP, - currency VARCHAR(3), - amount FLOAT - ) as VALUES - (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), - (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), - (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), - (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), - (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), - (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) - -# test_ordering_sensitive_aggregation -# ordering sensitive requirement should add a SortExec in the final plan. To satisfy amount ASC -# in the aggregation -statement ok -set datafusion.execution.target_partitions = 1; - -query TT -EXPLAIN SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts - FROM sales_global AS s - GROUP BY s.country ----- -logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount ASC NULLS LAST] AS amounts - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]]] - SubqueryAlias: s - TableScan: sales_global projection=[country, amount] -physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]@1 as amounts] - AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount)] - SortExec: expr=[amount@1 ASC NULLS LAST] - MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] - -query T? -SELECT s.country, (ARRAY_AGG(s.amount ORDER BY s.amount ASC)) AS amounts - FROM sales_global AS s - GROUP BY s.country ----- -GRC [30.0, 80.0] -FRA [50.0, 200.0] -TUR [75.0, 100.0] - -# test_ordering_sensitive_aggregation2 -# We should be able to satisfy the finest requirement among all aggregators, when we have multiple aggregators. -# Hence final plan should have SortExec: expr=[amount@1 DESC] to satisfy array_agg requirement. -query TT -EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM sales_global AS s - GROUP BY s.country ----- -logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] - SubqueryAlias: s - TableScan: sales_global projection=[country, amount] -physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] - AggregateExec: mode=Final, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] - CoalescePartitionsExec - AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] - SortExec: expr=[amount@1 DESC] - MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] - -query T?R -SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM sales_global AS s - GROUP BY s.country ----- -FRA [200.0, 50.0] 250 -TUR [100.0, 75.0] 175 -GRC [80.0, 30.0] 110 - -# test_ordering_sensitive_aggregation3 -# When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. -# test below should raise Plan Error. -statement error DataFusion error: Error during planning: Conflicting ordering requirements in aggregate functions -SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, - ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 - FROM sales_global AS s - GROUP BY s.country - -# test_ordering_sensitive_aggregation4 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to -# the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. -query TT -EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country ----- -logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] - SubqueryAlias: s - Sort: sales_global.country ASC NULLS LAST - TableScan: sales_global projection=[country, amount] -physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] - AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered - SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] - MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] - -query T?R -SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country ----- -FRA [200.0, 50.0] 250 -GRC [80.0, 30.0] 110 -TUR [100.0, 75.0] 175 - -# test_ordering_sensitive_aggregation5 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to -# the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in PartiallyOrdered mode. -query TT -EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country, s.zip_code ----- -logical_plan -Projection: s.country, s.zip_code, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] - SubqueryAlias: s - 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, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] - AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered - SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] - MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] - -query TI?R -SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country, s.zip_code ----- -FRA 1 [200.0, 50.0] 250 -GRC 0 [80.0, 30.0] 110 -TUR 1 [100.0, 75.0] 175 - -# test_ordering_sensitive_aggregation6 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to -# the existing ordering. When group by expressions contain aggregation requirement, we shouldn't append redundant expression. -# Hence in the final plan SortExec should be SortExec: expr=[country@0 DESC] not SortExec: expr=[country@0 ASC NULLS LAST,country@0 DESC] -query TT -EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country ----- -logical_plan -Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 - Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(s.amount)]] - SubqueryAlias: s - Sort: sales_global.country ASC NULLS LAST - TableScan: sales_global projection=[country, amount] -physical_plan -ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] - AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered - SortPreservingMergeExec: [country@0 ASC NULLS LAST] - SortExec: expr=[country@0 ASC NULLS LAST] - MemoryExec: partitions=4, partition_sizes=[1, 0, 0, 0] - -query T?R -SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, - SUM(s.amount) AS sum1 - FROM (SELECT * - FROM sales_global - ORDER BY country) AS s - GROUP BY s.country ----- -FRA [200.0, 50.0] 250 -GRC [80.0, 30.0] 110 -TUR [100.0, 75.0] 175 diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index b565b0dc9a7b..de7caaf92696 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2010,3 +2010,201 @@ SELECT a, d, statement ok drop table annotated_data_infinite2; + +# create a table for testing +statement ok +CREATE TABLE sales_global (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT + ) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), + (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) + +# test_ordering_sensitive_aggregation +# ordering sensitive requirement should add a SortExec in the final plan. To satisfy amount ASC +# in the aggregation +statement ok +set datafusion.execution.target_partitions = 1; + +statement ok +set datafusion.optimizer.skip_failed_rules = false; + +query TT +EXPLAIN SELECT country, (ARRAY_AGG(amount ORDER BY amount ASC)) AS amounts + FROM sales_global + GROUP BY country +---- +logical_plan +Projection: sales_global.country, ARRAYAGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts + Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAYAGG(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, ARRAYAGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] + AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(sales_global.amount)] + SortExec: expr=[amount@1 ASC NULLS LAST] + MemoryExec: partitions=1, partition_sizes=[1] + + +query T? +SELECT country, (ARRAY_AGG(amount ORDER BY amount ASC)) AS amounts + FROM sales_global + GROUP BY country +---- +GRC [30.0, 80.0] +FRA [50.0, 200.0] +TUR [75.0, 100.0] + +# test_ordering_sensitive_aggregation2 +# We should be able to satisfy the finest requirement among all aggregators, when we have multiple aggregators. +# Hence final plan should have SortExec: expr=[amount@1 DESC] to satisfy array_agg requirement. +query TT +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.country +---- +logical_plan +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] + SubqueryAlias: s + TableScan: sales_global projection=[country, amount] +physical_plan +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)] + SortExec: expr=[amount@1 DESC] + MemoryExec: partitions=1, partition_sizes=[1] + +query T?R +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM sales_global AS s + GROUP BY s.country +---- +FRA [200.0, 50.0] 250 +TUR [100.0, 75.0] 175 +GRC [80.0, 30.0] 110 + +# test_ordering_sensitive_aggregation3 +# When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. +# test below should raise Plan Error. +statement error DataFusion error: Error during planning: Conflicting ordering requirements in aggregate functions +SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, + ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 + FROM sales_global AS s + GROUP BY s.country + +# test_ordering_sensitive_aggregation4 +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. +# This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. +query TT +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country +---- +logical_plan +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] + SubqueryAlias: s + Sort: sales_global.country ASC NULLS LAST + TableScan: sales_global projection=[country, amount] +physical_plan +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered + SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] + MemoryExec: partitions=1, partition_sizes=[1] + +query T?R +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country +---- +FRA [200.0, 50.0] 250 +GRC [80.0, 30.0] 110 +TUR [100.0, 75.0] 175 + +# test_ordering_sensitive_aggregation5 +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. +# This test checks for whether we can satisfy aggregation requirement in PartiallyOrdered mode. +query TT +EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country, s.zip_code +---- +logical_plan +Projection: s.country, s.zip_code, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(s.amount)]] + SubqueryAlias: s + 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, ARRAYAGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] + AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered + SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] + MemoryExec: partitions=1, partition_sizes=[1] + +query TI?R +SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country, s.zip_code +---- +FRA 1 [200.0, 50.0] 250 +GRC 0 [80.0, 30.0] 110 +TUR 1 [100.0, 75.0] 175 + +# test_ordering_sensitive_aggregation6 +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# the existing ordering. When group by expressions contain aggregation requirement, we shouldn't append redundant expression. +# Hence in the final plan SortExec should be SortExec: expr=[country@0 DESC] not SortExec: expr=[country@0 ASC NULLS LAST,country@0 DESC] +query TT +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country +---- +logical_plan +Projection: s.country, ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 + Aggregate: groupBy=[[s.country]], aggr=[[ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(s.amount)]] + SubqueryAlias: s + Sort: sales_global.country ASC NULLS LAST + TableScan: sales_global projection=[country, amount] +physical_plan +ProjectionExec: expr=[country@0 as country, ARRAYAGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] + AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAYAGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered + SortExec: expr=[country@0 ASC NULLS LAST] + MemoryExec: partitions=1, partition_sizes=[1] + +query T?R +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + SUM(s.amount) AS sum1 + FROM (SELECT * + FROM sales_global + ORDER BY country) AS s + GROUP BY s.country +---- +FRA [200.0, 50.0] 250 +GRC [80.0, 30.0] 110 +TUR [100.0, 75.0] 175 diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index f73880340ec7..230e2fb916fa 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -1068,12 +1068,16 @@ impl fmt::Debug for Expr { fun, ref args, filter, + order_by, .. }) => { fmt_function(f, &fun.name, false, args, false)?; if let Some(fe) = filter { write!(f, " FILTER (WHERE {fe})")?; } + if let Some(ob) = order_by { + write!(f, " ORDER BY {:?}", ob)?; + } Ok(()) } Expr::Between(Between { diff --git a/datafusion/expr/src/tree_node/expr.rs b/datafusion/expr/src/tree_node/expr.rs index 493cdda55e5b..3c9884410912 100644 --- a/datafusion/expr/src/tree_node/expr.rs +++ b/datafusion/expr/src/tree_node/expr.rs @@ -301,7 +301,7 @@ impl TreeNode for Expr { transform_vec(args, &mut transform)?, distinct, transform_option_box(filter, &mut transform)?, - order_by, + transform_option_vec(order_by, &mut transform)?, )), Expr::GroupingSet(grouping_set) => match grouping_set { GroupingSet::Rollup(exprs) => Expr::GroupingSet(GroupingSet::Rollup( @@ -334,7 +334,7 @@ impl TreeNode for Expr { fun, transform_vec(args, &mut transform)?, transform_option_box(filter, &mut transform)?, - order_by, + transform_option_vec(order_by, &mut transform)?, )) } Expr::InList(InList { @@ -387,6 +387,21 @@ where .transpose() } +/// &mut transform a Option<`Vec` of `Expr`s> +fn transform_option_vec( + option_box: Option>, + transform: &mut F, +) -> Result>> +where + F: FnMut(Expr) -> Result, +{ + Ok(if let Some(exprs) = option_box { + Some(transform_vec(exprs, transform)?) + } else { + None + }) +} + /// &mut transform a `Vec` of `Expr`s fn transform_vec(v: Vec, transform: &mut F) -> Result> where From 1d8d4baa5fbdbf28067cb98c5735e72e20044e9a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 May 2023 16:36:34 +0300 Subject: [PATCH 22/24] Simplifications, Address reviews --- .../core/src/physical_plan/aggregates/mod.rs | 12 ++++-------- .../tests/sqllogictests/test_files/groupby.slt | 15 +++++++++++---- datafusion/sql/src/expr/mod.rs | 1 + 3 files changed, 16 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 7d1f445bb846..9f3ba7336afc 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -379,8 +379,8 @@ fn get_finest_requirement< // If neither of the requirements satisfy the other, this means // requirements are conflicting. Currently, we do not support // conflicting requirements. - return Err(DataFusionError::Plan( - "Conflicting ordering requirements in aggregate functions".to_string(), + return Err(DataFusionError::NotImplemented( + "Conflicting ordering requirements in aggregate functions is not supported".to_string(), )); } else { result = Some(fn_reqs.clone()); @@ -419,12 +419,8 @@ impl AggregateExec { || input.equivalence_properties(), || input.ordering_equivalence_properties(), )?; - aggregator_requirement = requirement.map(|exprs| { - exprs - .into_iter() - .map(PhysicalSortRequirement::from) - .collect::>() - }); + aggregator_requirement = requirement + .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); } // construct a map from the input columns to the output columns of the Aggregation diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index de7caaf92696..023740144d9c 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2033,9 +2033,6 @@ CREATE TABLE sales_global (zip_code INT, statement ok set datafusion.execution.target_partitions = 1; -statement ok -set datafusion.optimizer.skip_failed_rules = false; - query TT EXPLAIN SELECT country, (ARRAY_AGG(amount ORDER BY amount ASC)) AS amounts FROM sales_global @@ -2091,10 +2088,20 @@ FRA [200.0, 50.0] 250 TUR [100.0, 75.0] 175 GRC [80.0, 30.0] 110 +# test_ordering_sensitive_multiple_req +# Currently we do not support multiple ordering requirement for aggregation +# once this support is added. This test should change +# See issue: https://github.com/sqlparser-rs/sqlparser-rs/issues/875 +statement error DataFusion error: SQL error: ParserError\("Expected \), found: ,"\) +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC, s.country DESC) AS amounts, + SUM(s.amount ORDER BY s.amount DESC) AS sum1 + FROM sales_global AS s + GROUP BY s.country + # test_ordering_sensitive_aggregation3 # When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. # test below should raise Plan Error. -statement error DataFusion error: Error during planning: Conflicting ordering requirements in aggregate functions +statement error DataFusion error: This feature is not implemented: Conflicting ordering requirements in aggregate functions is not supported SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 36989b530bf9..b914149cf4e1 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -323,6 +323,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let order_by = if let Some(order_by) = order_by { // TODO: Once sqlparser supports multiple order by clause, handle it + // see issue: https://github.com/sqlparser-rs/sqlparser-rs/issues/875 Some(vec![self.order_by_to_sort_expr( *order_by, input_schema, From 01c8da800806f5d13fbeabf52d30bfd882fa8d21 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 May 2023 17:06:38 +0300 Subject: [PATCH 23/24] Update comment --- datafusion/core/tests/sqllogictests/test_files/groupby.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 023740144d9c..83bf40987eb0 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2109,7 +2109,7 @@ SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country # test_ordering_sensitive_aggregation4 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. # This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. query TT From c23945d39fdf628c35c3aaaac282aee75ee8dce5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 May 2023 17:31:16 +0300 Subject: [PATCH 24/24] Resolve linter errors --- datafusion/core/src/physical_plan/aggregates/mod.rs | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 9f3ba7336afc..247dfc27784e 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -455,12 +455,10 @@ impl AggregateExec { } else { vec![] }; - let mut requirement = requirement_prefix - .into_iter() - .map(PhysicalSortRequirement::from) - .collect::>(); + let mut requirement = + PhysicalSortRequirement::from_sort_exprs(requirement_prefix.iter()); for req in aggregator_requirement { - if requirement.iter().all(|item| req.expr().ne(item.expr())) { + if requirement.iter().all(|item| req.expr.ne(&item.expr)) { requirement.push(req); } }