From 7411d6b7ce8f3bc26ffcda11e14e543efe40fd64 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 3 May 2023 10:11:51 +0300 Subject: [PATCH 01/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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/53] 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 6fd96da83ae67d23aecb57f706981ddf271c01b7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 9 May 2023 16:55:35 +0300 Subject: [PATCH 18/53] initial commit, add test --- datafusion/common/Cargo.toml | 6 ++- datafusion/core/Cargo.toml | 3 +- datafusion/core/tests/sql/group_by.rs | 53 +++++++++++++++++++++++++++ datafusion/expr/Cargo.toml | 6 ++- datafusion/sql/Cargo.toml | 6 ++- datafusion/sql/src/expr/mod.rs | 50 +++++++++++++++++++++++++ 6 files changed, 117 insertions(+), 7 deletions(-) diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 701cbdb51510..c6590ea9861a 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -47,7 +47,9 @@ num_cpus = "1.13.0" object_store = { version = "0.5.4", default-features = false, optional = true } parquet = { workspace = true, optional = true } pyo3 = { version = "0.18.0", optional = true } -sqlparser = "0.33" - +# sqlparser = "0.33" +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de"} +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} [dev-dependencies] rand = "0.8.4" diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index f24e200e26c9..37c2a1f7417b 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -85,7 +85,8 @@ percent-encoding = "2.2.0" pin-project-lite = "^0.2.7" rand = "0.8" smallvec = { version = "1.6", features = ["union"] } -sqlparser = { version = "0.33", features = ["visitor"] } +# sqlparser = { version = "0.33", features = ["visitor"] } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"]} tempfile = "3" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } tokio-stream = "0.1" diff --git a/datafusion/core/tests/sql/group_by.rs b/datafusion/core/tests/sql/group_by.rs index 7a4d4d289700..57ae9506ea3b 100644 --- a/datafusion/core/tests/sql/group_by.rs +++ b/datafusion/core/tests/sql/group_by.rs @@ -257,3 +257,56 @@ async fn test_source_sorted_groupby2() -> Result<()> { assert_batches_eq!(expected, &actual); Ok(()) } + +#[tokio::test] +async fn test_source_sorted_groupby3() -> Result<()> { + let tmpdir = TempDir::new().unwrap(); + let session_config = SessionConfig::new().with_target_partitions(1); + let ctx = get_test_context2(&tmpdir, true, session_config).await?; + + let sql = "SELECT a, FIRST(c ORDER BY a DESC) + FROM annotated_data + GROUP BY a, b"; + // let sql = "SELECT a, ARRAY_AGG(c ORDER BY a DESC, b DESC) + // FROM annotated_data + // GROUP BY a, b"; + + 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(); + let expected = { + vec![ + "ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data.c)@2 as summation1]", + " AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data.c)], ordering_mode=PartiallyOrdered", + ] + }; + + 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![ + "+---+---+------------+", + "| a | d | summation1 |", + "+---+---+------------+", + "| 0 | 0 | 292 |", + "| 0 | 2 | 196 |", + "| 0 | 1 | 315 |", + "| 0 | 4 | 164 |", + "| 0 | 3 | 258 |", + "| 1 | 0 | 622 |", + "| 1 | 3 | 299 |", + "| 1 | 1 | 1043 |", + "| 1 | 4 | 913 |", + "| 1 | 2 | 848 |", + "+---+---+------------+", + ]; + assert_batches_eq!(expected, &actual); + Ok(()) +} diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 221188ce37c9..8062ddf27a1e 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -38,8 +38,10 @@ path = "src/lib.rs" ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } arrow = { workspace = true } datafusion-common = { path = "../common", version = "23.0.0" } -sqlparser = "0.33" - +# sqlparser = "0.33" +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de"} +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} [dev-dependencies] ctor = "0.2.0" env_logger = "0.10" diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index d99d4a515b31..8f86b649dab8 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -42,8 +42,10 @@ arrow-schema = { workspace = true } datafusion-common = { path = "../common", version = "23.0.0" } datafusion-expr = { path = "../expr", version = "23.0.0" } log = "^0.4" -sqlparser = "0.33" - +# sqlparser = "0.33" +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de"} +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} [dev-dependencies] ctor = "0.2.0" env_logger = "0.10" diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index cc3f3140c8cd..b0f006b44035 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -295,6 +295,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Subquery(subquery) => self.parse_scalar_subquery(*subquery, schema, planner_context), SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), + SQLExpr::FIRST(first_agg) => self.parse_first_agg(first_agg, schema, planner_context), + SQLExpr::LAST(last_agg) => todo!(), _ => Err(DataFusionError::NotImplemented(format!( "Unsupported ast node in sqltorel: {sql:?}" @@ -350,6 +352,54 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ))) } + fn parse_first_agg( + &self, + array_agg: FirstAgg, + input_schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + // Some dialects have special syntax for array_agg. DataFusion only supports it like a function. + let ArrayAgg { + distinct, + expr, + order_by, + limit, + within_group, + } = array_agg; + + let order_by = if let Some(order_by) = order_by { + // 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 + }; + + if let Some(limit) = limit { + return Err(DataFusionError::NotImplemented(format!( + "LIMIT not supported in ARRAY_AGG: {limit}" + ))); + } + + if within_group { + return Err(DataFusionError::NotImplemented( + "WITHIN GROUP not supported in ARRAY_AGG".to_string(), + )); + } + + 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, order_by, + ))) + } + fn sql_in_list_to_expr( &self, expr: SQLExpr, From 61cc1099f829b4cc4a0802b65c11582c109893da Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 9 May 2023 17:46:40 +0300 Subject: [PATCH 19/53] Add support for FIRST Aggregate function. --- datafusion/common/Cargo.toml | 2 +- datafusion/core/Cargo.toml | 2 +- datafusion/core/tests/sql/group_by.rs | 26 ++- datafusion/expr/Cargo.toml | 2 +- datafusion/expr/src/aggregate_function.rs | 6 +- .../expr/src/type_coercion/aggregates.rs | 2 +- .../physical-expr/src/aggregate/build_in.rs | 5 + .../physical-expr/src/aggregate/first_last.rs | 161 ++++++++++++++++++ datafusion/physical-expr/src/aggregate/mod.rs | 1 + .../physical-expr/src/expressions/mod.rs | 1 + datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 1 + datafusion/proto/src/logical_plan/to_proto.rs | 2 + datafusion/sql/Cargo.toml | 2 +- datafusion/sql/src/expr/mod.rs | 12 +- 17 files changed, 204 insertions(+), 28 deletions(-) create mode 100644 datafusion/physical-expr/src/aggregate/first_last.rs diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index c6590ea9861a..cfcd8c14fd0c 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -49,7 +49,7 @@ parquet = { workspace = true, optional = true } pyo3 = { version = "0.18.0", optional = true } # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de"} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} [dev-dependencies] rand = "0.8.4" diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 37c2a1f7417b..31a839cd2440 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -86,7 +86,7 @@ pin-project-lite = "^0.2.7" rand = "0.8" smallvec = { version = "1.6", features = ["union"] } # sqlparser = { version = "0.33", features = ["visitor"] } -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"]} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"] } tempfile = "3" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } tokio-stream = "0.1" diff --git a/datafusion/core/tests/sql/group_by.rs b/datafusion/core/tests/sql/group_by.rs index 57ae9506ea3b..5465b65aaade 100644 --- a/datafusion/core/tests/sql/group_by.rs +++ b/datafusion/core/tests/sql/group_by.rs @@ -277,8 +277,8 @@ async fn test_source_sorted_groupby3() -> Result<()> { let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let expected = { vec![ - "ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data.c)@2 as summation1]", - " AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data.c)], ordering_mode=PartiallyOrdered", + "ProjectionExec: expr=[a@0 as a, FIRST(annotated_data.c) ORDER BY [a DESC NULLS FIRST]@2 as FIRST(annotated_data.c)]", + " AggregateExec: mode=Single, gby=[a@1 as a, b@2 as b], aggr=[FIRST(annotated_data.c)], ordering_mode=FullyOrdered", ] }; @@ -292,20 +292,14 @@ async fn test_source_sorted_groupby3() -> Result<()> { let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+---+---+------------+", - "| a | d | summation1 |", - "+---+---+------------+", - "| 0 | 0 | 292 |", - "| 0 | 2 | 196 |", - "| 0 | 1 | 315 |", - "| 0 | 4 | 164 |", - "| 0 | 3 | 258 |", - "| 1 | 0 | 622 |", - "| 1 | 3 | 299 |", - "| 1 | 1 | 1043 |", - "| 1 | 4 | 913 |", - "| 1 | 2 | 848 |", - "+---+---+------------+", + "+---+-------------------------+", + "| a | FIRST(annotated_data.c) |", + "+---+-------------------------+", + "| 0 | 0 |", + "| 0 | 25 |", + "| 1 | 50 |", + "| 1 | 75 |", + "+---+-------------------------+", ]; assert_batches_eq!(expected, &actual); Ok(()) diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 8062ddf27a1e..896956d76629 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -40,7 +40,7 @@ arrow = { workspace = true } datafusion-common = { path = "../common", version = "23.0.0" } # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de"} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} [dev-dependencies] ctor = "0.2.0" diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index dd27d9504bee..21b7f3628a7a 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -42,6 +42,8 @@ pub enum AggregateFunction { ApproxDistinct, /// array_agg ArrayAgg, + /// first + First, /// Variance (Sample) Variance, /// Variance (Population) @@ -155,6 +157,7 @@ pub fn return_type( coerced_data_types[0].clone(), true, )))), + AggregateFunction::First => Ok(coerced_data_types[0].clone()), AggregateFunction::ApproxPercentileCont => Ok(coerced_data_types[0].clone()), AggregateFunction::ApproxPercentileContWithWeight => { Ok(coerced_data_types[0].clone()) @@ -205,7 +208,8 @@ pub fn signature(fun: &AggregateFunction) -> Signature { | AggregateFunction::Stddev | AggregateFunction::StddevPop | AggregateFunction::Median - | AggregateFunction::ApproxMedian => { + | AggregateFunction::ApproxMedian + | AggregateFunction::First => { Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable) } AggregateFunction::Covariance | AggregateFunction::CovariancePop => { diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 26498ed5b6a3..2eb48bf47be5 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -215,7 +215,7 @@ pub fn coerce_types( } Ok(input_types.to_vec()) } - AggregateFunction::Median => Ok(input_types.to_vec()), + AggregateFunction::Median | AggregateFunction::First => Ok(input_types.to_vec()), AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]), } } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index d3b85d706d7b..fc300c18cdad 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -98,6 +98,11 @@ pub fn create_aggregate_expr( input_phy_types[0].clone(), )) } + (AggregateFunction::First, _) => Arc::new(expressions::FirstAgg::new( + input_phy_exprs[0].clone(), + name, + input_phy_types[0].clone(), + )), (AggregateFunction::Min, _) => Arc::new(expressions::Min::new( input_phy_exprs[0].clone(), name, diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs new file mode 100644 index 000000000000..91d34224ed68 --- /dev/null +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -0,0 +1,161 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Defines physical expressions that can evaluated at runtime during query execution + +use crate::aggregate::utils::down_cast_any_ref; +use crate::expressions::format_state_name; +use crate::{AggregateExpr, PhysicalExpr}; +use arrow::array::ArrayRef; +use arrow::datatypes::{DataType, Field}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::Accumulator; +use std::any::Any; +use std::sync::Arc; + +/// FIRST aggregate expression +#[derive(Debug)] +pub struct FirstAgg { + name: String, + pub data_type: DataType, + expr: Arc, +} + +impl FirstAgg { + /// Create a new ArrayAgg aggregate function + pub fn new( + expr: Arc, + name: impl Into, + data_type: DataType, + ) -> Self { + Self { + name: name.into(), + data_type, + expr, + } + } +} + +impl AggregateExpr for FirstAgg { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result { + Ok(Field::new(&self.name, self.data_type.clone(), true)) + } + + fn create_accumulator(&self) -> Result> { + Ok(Box::new(FirstAccumulator::try_new(&self.data_type)?)) + } + + fn state_fields(&self) -> Result> { + Ok(vec![Field::new( + format_state_name(&self.name, "first"), + self.data_type.clone(), + true, + )]) + } + + fn expressions(&self) -> Vec> { + vec![self.expr.clone()] + } + + fn name(&self) -> &str { + &self.name + } + + fn supports_bounded_execution(&self) -> bool { + true + } + + // TODO: Add support for reverse expr + + fn create_sliding_accumulator(&self) -> Result> { + Ok(Box::new(FirstAccumulator::try_new(&self.data_type)?)) + } +} + +impl PartialEq for FirstAgg { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.name == x.name + && self.data_type == x.data_type + && self.expr.eq(&x.expr) + }) + .unwrap_or(false) + } +} + +#[derive(Debug)] +struct FirstAccumulator { + first: ScalarValue, + count: u64, +} + +impl FirstAccumulator { + /// new First accumulator + pub fn try_new(data_type: &DataType) -> Result { + Ok(Self { + first: ScalarValue::try_from(data_type)?, + count: 0, + }) + } +} + +impl Accumulator for FirstAccumulator { + fn state(&self) -> Result> { + Ok(vec![self.first.clone(), ScalarValue::from(self.count)]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + // If we have seen first value, we shouldn't update it + let values = &values[0]; + if self.count == 0 && values.len() > 0 { + self.first = ScalarValue::try_from_array(values, 0)?; + } + self.count += (values.len() - values.null_count()) as u64; + Ok(()) + } + + fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = &values[0]; + self.count -= (values.len() - values.null_count()) as u64; + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // FIRST(first1, first2, first3, ...) = sum1 + sum2 + sum3 + ... + self.update_batch(states) + } + + fn evaluate(&self) -> Result { + if self.count == 0 { + ScalarValue::try_from(&self.first.get_datatype()) + } else { + Ok(self.first.clone()) + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) - std::mem::size_of_val(&self.first) + + self.first.size() + } +} diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index b3e37a8f9246..9b307ff847b6 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -35,6 +35,7 @@ pub(crate) mod correlation; pub(crate) mod count; pub(crate) mod count_distinct; pub(crate) mod covariance; +pub(crate) mod first_last; pub(crate) mod grouping; pub(crate) mod median; #[macro_use] diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 135e24dc8355..ca2c94e28eee 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -52,6 +52,7 @@ pub use crate::aggregate::correlation::Correlation; pub use crate::aggregate::count::Count; pub use crate::aggregate::count_distinct::DistinctCount; pub use crate::aggregate::covariance::{Covariance, CovariancePop}; +pub use crate::aggregate::first_last::FirstAgg; pub use crate::aggregate::grouping::Grouping; pub use crate::aggregate::median::Median; pub use crate::aggregate::min_max::{Max, Min}; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 3ed7361b0d29..93b2ad04e350 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -561,6 +561,7 @@ enum AggregateFunction { APPROX_PERCENTILE_CONT_WITH_WEIGHT = 16; GROUPING = 17; MEDIAN = 18; + FIRST = 19; } message AggregateExprNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 5baec89725b6..742097e30c66 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -458,6 +458,7 @@ impl serde::Serialize for AggregateFunction { Self::ApproxPercentileContWithWeight => "APPROX_PERCENTILE_CONT_WITH_WEIGHT", Self::Grouping => "GROUPING", Self::Median => "MEDIAN", + Self::First => "FIRST", }; serializer.serialize_str(variant) } @@ -488,6 +489,7 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "APPROX_PERCENTILE_CONT_WITH_WEIGHT", "GROUPING", "MEDIAN", + "FIRST", ]; struct GeneratedVisitor; @@ -549,6 +551,7 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "APPROX_PERCENTILE_CONT_WITH_WEIGHT" => Ok(AggregateFunction::ApproxPercentileContWithWeight), "GROUPING" => Ok(AggregateFunction::Grouping), "MEDIAN" => Ok(AggregateFunction::Median), + "FIRST" => Ok(AggregateFunction::First), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 6dafdf3e0c58..21aabf8785a4 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2384,6 +2384,7 @@ pub enum AggregateFunction { ApproxPercentileContWithWeight = 16, Grouping = 17, Median = 18, + First = 19, } impl AggregateFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2413,6 +2414,7 @@ impl AggregateFunction { } AggregateFunction::Grouping => "GROUPING", AggregateFunction::Median => "MEDIAN", + AggregateFunction::First => "FIRST", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2439,6 +2441,7 @@ impl AggregateFunction { } "GROUPING" => Some(Self::Grouping), "MEDIAN" => Some(Self::Median), + "FIRST" => Some(Self::First), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 8d1bcdb8ac40..297ca6ec550d 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -520,6 +520,7 @@ impl From for AggregateFunction { protobuf::AggregateFunction::ApproxMedian => Self::ApproxMedian, protobuf::AggregateFunction::Grouping => Self::Grouping, protobuf::AggregateFunction::Median => Self::Median, + protobuf::AggregateFunction::First => Self::First, } } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 847b933b981c..d849e6f90769 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -382,6 +382,7 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::ApproxMedian => Self::ApproxMedian, AggregateFunction::Grouping => Self::Grouping, AggregateFunction::Median => Self::Median, + AggregateFunction::First => Self::First, } } } @@ -656,6 +657,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, AggregateFunction::Median => protobuf::AggregateFunction::Median, + AggregateFunction::First => protobuf::AggregateFunction::First, }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index 8f86b649dab8..9d1d507f1019 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -44,7 +44,7 @@ datafusion-expr = { path = "../expr", version = "23.0.0" } log = "^0.4" # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de"} +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} [dev-dependencies] ctor = "0.2.0" diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index b0f006b44035..d5cf4583aebc 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -34,7 +34,7 @@ use datafusion_expr::{ col, expr, lit, AggregateFunction, Between, BinaryExpr, BuiltinScalarFunction, Cast, Expr, ExprSchemable, GetIndexedField, Like, Operator, TryCast, }; -use sqlparser::ast::{ArrayAgg, Expr as SQLExpr, TrimWhereField, Value}; +use sqlparser::ast::{ArrayAgg, Expr as SQLExpr, FirstAgg, TrimWhereField, Value}; use sqlparser::parser::ParserError::ParserError; impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -296,7 +296,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), SQLExpr::FIRST(first_agg) => self.parse_first_agg(first_agg, schema, planner_context), - SQLExpr::LAST(last_agg) => todo!(), + SQLExpr::LAST(_last_agg) => todo!(), _ => Err(DataFusionError::NotImplemented(format!( "Unsupported ast node in sqltorel: {sql:?}" @@ -359,7 +359,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { planner_context: &mut PlannerContext, ) -> Result { // Some dialects have special syntax for array_agg. DataFusion only supports it like a function. - let ArrayAgg { + let FirstAgg { distinct, expr, order_by, @@ -380,13 +380,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if let Some(limit) = limit { return Err(DataFusionError::NotImplemented(format!( - "LIMIT not supported in ARRAY_AGG: {limit}" + "LIMIT not supported in FIRST: {limit}" ))); } if within_group { return Err(DataFusionError::NotImplemented( - "WITHIN GROUP not supported in ARRAY_AGG".to_string(), + "WITHIN GROUP not supported in FIRST".to_string(), )); } @@ -394,7 +394,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { vec![self.sql_expr_to_logical_expr(*expr, input_schema, planner_context)?]; // next, aggregate built-ins - let fun = AggregateFunction::ArrayAgg; + let fun = AggregateFunction::First; Ok(Expr::AggregateFunction(expr::AggregateFunction::new( fun, args, distinct, None, order_by, ))) From 682b1f4eb5e39435eb0faa636810dcce30d34fbf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 10 May 2023 12:48:22 +0300 Subject: [PATCH 20/53] Add support for last aggregate --- datafusion/core/tests/sql/group_by.rs | 67 +++++++-- .../core/tests/user_defined_aggregates.rs | 8 +- datafusion/expr/src/aggregate_function.rs | 9 +- .../expr/src/type_coercion/aggregates.rs | 4 +- .../physical-expr/src/aggregate/build_in.rs | 15 +- .../physical-expr/src/aggregate/first_last.rs | 134 ++++++++++++++++++ .../physical-expr/src/expressions/mod.rs | 2 +- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 3 + datafusion/proto/src/generated/prost.rs | 3 + .../proto/src/logical_plan/from_proto.rs | 1 + datafusion/proto/src/logical_plan/to_proto.rs | 2 + datafusion/sql/src/expr/mod.rs | 58 +++++++- 13 files changed, 277 insertions(+), 30 deletions(-) diff --git a/datafusion/core/tests/sql/group_by.rs b/datafusion/core/tests/sql/group_by.rs index 5465b65aaade..d7b87b2c87fb 100644 --- a/datafusion/core/tests/sql/group_by.rs +++ b/datafusion/core/tests/sql/group_by.rs @@ -264,12 +264,9 @@ async fn test_source_sorted_groupby3() -> Result<()> { let session_config = SessionConfig::new().with_target_partitions(1); let ctx = get_test_context2(&tmpdir, true, session_config).await?; - let sql = "SELECT a, FIRST(c ORDER BY a DESC) + let sql = "SELECT a, b, FIRST(c ORDER BY a DESC) as first_c FROM annotated_data GROUP BY a, b"; - // let sql = "SELECT a, ARRAY_AGG(c ORDER BY a DESC, b DESC) - // FROM annotated_data - // GROUP BY a, b"; let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); @@ -277,7 +274,7 @@ async fn test_source_sorted_groupby3() -> Result<()> { let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let expected = { vec![ - "ProjectionExec: expr=[a@0 as a, FIRST(annotated_data.c) ORDER BY [a DESC NULLS FIRST]@2 as FIRST(annotated_data.c)]", + "ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST(annotated_data.c) ORDER BY [a DESC NULLS FIRST]@2 as first_c]", " AggregateExec: mode=Single, gby=[a@1 as a, b@2 as b], aggr=[FIRST(annotated_data.c)], ordering_mode=FullyOrdered", ] }; @@ -292,14 +289,58 @@ async fn test_source_sorted_groupby3() -> Result<()> { let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+---+-------------------------+", - "| a | FIRST(annotated_data.c) |", - "+---+-------------------------+", - "| 0 | 0 |", - "| 0 | 25 |", - "| 1 | 50 |", - "| 1 | 75 |", - "+---+-------------------------+", + "+---+---+---------+", + "| a | b | first_c |", + "+---+---+---------+", + "| 0 | 0 | 0 |", + "| 0 | 1 | 25 |", + "| 1 | 2 | 50 |", + "| 1 | 3 | 75 |", + "+---+---+---------+", + ]; + assert_batches_eq!(expected, &actual); + Ok(()) +} + +#[tokio::test] +async fn test_source_sorted_groupby4() -> Result<()> { + let tmpdir = TempDir::new().unwrap(); + let session_config = SessionConfig::new().with_target_partitions(1); + let ctx = get_test_context2(&tmpdir, true, session_config).await?; + + let sql = "SELECT a, b, LAST(c ORDER BY a DESC) as last_c + FROM annotated_data + GROUP BY a, b"; + + 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(); + let expected = { + vec![ + "ProjectionExec: expr=[a@0 as a, b@1 as b, LAST(annotated_data.c) ORDER BY [a DESC NULLS FIRST]@2 as last_c]", + " AggregateExec: mode=Single, gby=[a@1 as a, b@2 as b], aggr=[LAST(annotated_data.c)], ordering_mode=FullyOrdered", + ] + }; + + 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![ + "+---+---+--------+", + "| a | b | last_c |", + "+---+---+--------+", + "| 0 | 0 | 24 |", + "| 0 | 1 | 49 |", + "| 1 | 2 | 74 |", + "| 1 | 3 | 99 |", + "+---+---+--------+", ]; assert_batches_eq!(expected, &actual); Ok(()) diff --git a/datafusion/core/tests/user_defined_aggregates.rs b/datafusion/core/tests/user_defined_aggregates.rs index 1047f73df4cd..b998fc355fa6 100644 --- a/datafusion/core/tests/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined_aggregates.rs @@ -43,10 +43,10 @@ use datafusion_common::cast::as_primitive_array; /// Basic query for with a udaf returning a structure async fn test_udf_returning_struct() { let ctx = udaf_struct_context(); - let sql = "SELECT first(value, time) from t"; + let sql = "SELECT first_value(value, time) from t"; let expected = vec![ "+------------------------------------------------+", - "| first(t.value,t.time) |", + "| first_value(t.value,t.time) |", "+------------------------------------------------+", "| {value: 2.0, time: 1970-01-01T00:00:00.000002} |", "+------------------------------------------------+", @@ -58,7 +58,7 @@ async fn test_udf_returning_struct() { /// Demonstrate extracting the fields from the a structure using a subquery async fn test_udf_returning_struct_sq() { let ctx = udaf_struct_context(); - let sql = "select sq.first['value'], sq.first['time'] from (SELECT first(value, time) as first from t) as sq"; + let sql = "select sq.first['value'], sq.first['time'] from (SELECT first_value(value, time) as first from t) as sq"; let expected = vec![ "+-----------------+----------------------------+", "| sq.first[value] | sq.first[time] |", @@ -117,7 +117,7 @@ fn register_aggregate(ctx: &mut SessionContext) { let volatility = Volatility::Immutable; - let name = "first"; + let name = "first_value"; let first = AggregateUDF::new( name, diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 21b7f3628a7a..65546705448a 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -44,6 +44,8 @@ pub enum AggregateFunction { ArrayAgg, /// first First, + /// last + Last, /// Variance (Sample) Variance, /// Variance (Population) @@ -157,7 +159,6 @@ pub fn return_type( coerced_data_types[0].clone(), true, )))), - AggregateFunction::First => Ok(coerced_data_types[0].clone()), AggregateFunction::ApproxPercentileCont => Ok(coerced_data_types[0].clone()), AggregateFunction::ApproxPercentileContWithWeight => { Ok(coerced_data_types[0].clone()) @@ -166,6 +167,9 @@ pub fn return_type( Ok(coerced_data_types[0].clone()) } AggregateFunction::Grouping => Ok(DataType::Int32), + AggregateFunction::First | AggregateFunction::Last => { + Ok(coerced_data_types[0].clone()) + } } } @@ -209,7 +213,8 @@ pub fn signature(fun: &AggregateFunction) -> Signature { | AggregateFunction::StddevPop | AggregateFunction::Median | AggregateFunction::ApproxMedian - | AggregateFunction::First => { + | AggregateFunction::First + | AggregateFunction::Last => { Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable) } AggregateFunction::Covariance | AggregateFunction::CovariancePop => { diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 2eb48bf47be5..1a887a0bc986 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -215,7 +215,9 @@ pub fn coerce_types( } Ok(input_types.to_vec()) } - AggregateFunction::Median | AggregateFunction::First => Ok(input_types.to_vec()), + AggregateFunction::Median + | AggregateFunction::First + | AggregateFunction::Last => Ok(input_types.to_vec()), AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]), } } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index fc300c18cdad..ae01ed72c6fa 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -98,11 +98,6 @@ pub fn create_aggregate_expr( input_phy_types[0].clone(), )) } - (AggregateFunction::First, _) => Arc::new(expressions::FirstAgg::new( - input_phy_exprs[0].clone(), - name, - input_phy_types[0].clone(), - )), (AggregateFunction::Min, _) => Arc::new(expressions::Min::new( input_phy_exprs[0].clone(), name, @@ -263,6 +258,16 @@ pub fn create_aggregate_expr( "MEDIAN(DISTINCT) aggregations are not available".to_string(), )); } + (AggregateFunction::First, _) => Arc::new(expressions::FirstAgg::new( + input_phy_exprs[0].clone(), + name, + input_phy_types[0].clone(), + )), + (AggregateFunction::Last, _) => Arc::new(expressions::LastAgg::new( + input_phy_exprs[0].clone(), + name, + input_phy_types[0].clone(), + )), }) } diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 91d34224ed68..fa4568d84a50 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -22,6 +22,7 @@ use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; +use arrow_array::Array; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Accumulator; use std::any::Any; @@ -159,3 +160,136 @@ impl Accumulator for FirstAccumulator { + self.first.size() } } + +/// LAST aggregate expression +#[derive(Debug)] +pub struct LastAgg { + name: String, + pub data_type: DataType, + expr: Arc, +} + +impl LastAgg { + /// Create a new ArrayAgg aggregate function + pub fn new( + expr: Arc, + name: impl Into, + data_type: DataType, + ) -> Self { + Self { + name: name.into(), + data_type, + expr, + } + } +} + +impl AggregateExpr for LastAgg { + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn field(&self) -> Result { + Ok(Field::new(&self.name, self.data_type.clone(), true)) + } + + fn create_accumulator(&self) -> Result> { + Ok(Box::new(LastAccumulator::try_new(&self.data_type)?)) + } + + fn state_fields(&self) -> Result> { + Ok(vec![Field::new( + format_state_name(&self.name, "last"), + self.data_type.clone(), + true, + )]) + } + + fn expressions(&self) -> Vec> { + vec![self.expr.clone()] + } + + fn name(&self) -> &str { + &self.name + } + + fn supports_bounded_execution(&self) -> bool { + true + } + + // TODO: Add support for reverse expr + + fn create_sliding_accumulator(&self) -> Result> { + Ok(Box::new(LastAccumulator::try_new(&self.data_type)?)) + } +} + +impl PartialEq for LastAgg { + fn eq(&self, other: &dyn Any) -> bool { + down_cast_any_ref(other) + .downcast_ref::() + .map(|x| { + self.name == x.name + && self.data_type == x.data_type + && self.expr.eq(&x.expr) + }) + .unwrap_or(false) + } +} + +#[derive(Debug)] +struct LastAccumulator { + last: ScalarValue, + count: u64, +} + +impl LastAccumulator { + /// new First accumulator + pub fn try_new(data_type: &DataType) -> Result { + Ok(Self { + last: ScalarValue::try_from(data_type)?, + count: 0, + }) + } +} + +impl Accumulator for LastAccumulator { + fn state(&self) -> Result> { + Ok(vec![self.last.clone(), ScalarValue::from(self.count)]) + } + + fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + // If we have seen first value, we shouldn't update it + let values = &values[0]; + if values.len() > 0 { + // Update with last value in the array. + self.last = ScalarValue::try_from_array(values, values.len() - 1)?; + } + self.count += (values.len() - values.null_count()) as u64; + Ok(()) + } + + fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + let values = &values[0]; + self.count -= (values.len() - values.null_count()) as u64; + Ok(()) + } + + fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { + // FIRST(first1, first2, first3, ...) = sum1 + sum2 + sum3 + ... + self.update_batch(states) + } + + fn evaluate(&self) -> Result { + if self.count == 0 { + ScalarValue::try_from(&self.last.get_datatype()) + } else { + Ok(self.last.clone()) + } + } + + fn size(&self) -> usize { + std::mem::size_of_val(self) - std::mem::size_of_val(&self.last) + self.last.size() + } +} diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index ca2c94e28eee..b20c978d8646 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -52,7 +52,7 @@ pub use crate::aggregate::correlation::Correlation; pub use crate::aggregate::count::Count; pub use crate::aggregate::count_distinct::DistinctCount; pub use crate::aggregate::covariance::{Covariance, CovariancePop}; -pub use crate::aggregate::first_last::FirstAgg; +pub use crate::aggregate::first_last::{FirstAgg, LastAgg}; pub use crate::aggregate::grouping::Grouping; pub use crate::aggregate::median::Median; pub use crate::aggregate::min_max::{Max, Min}; diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 93b2ad04e350..18e3390a35d0 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -562,6 +562,7 @@ enum AggregateFunction { GROUPING = 17; MEDIAN = 18; FIRST = 19; + LAST = 20; } message AggregateExprNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 742097e30c66..97f00a3cbac0 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -459,6 +459,7 @@ impl serde::Serialize for AggregateFunction { Self::Grouping => "GROUPING", Self::Median => "MEDIAN", Self::First => "FIRST", + Self::Last => "LAST", }; serializer.serialize_str(variant) } @@ -490,6 +491,7 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "GROUPING", "MEDIAN", "FIRST", + "LAST", ]; struct GeneratedVisitor; @@ -552,6 +554,7 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "GROUPING" => Ok(AggregateFunction::Grouping), "MEDIAN" => Ok(AggregateFunction::Median), "FIRST" => Ok(AggregateFunction::First), + "LAST" => Ok(AggregateFunction::Last), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 21aabf8785a4..94805ca19f63 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2385,6 +2385,7 @@ pub enum AggregateFunction { Grouping = 17, Median = 18, First = 19, + Last = 20, } impl AggregateFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2415,6 +2416,7 @@ impl AggregateFunction { AggregateFunction::Grouping => "GROUPING", AggregateFunction::Median => "MEDIAN", AggregateFunction::First => "FIRST", + AggregateFunction::Last => "LAST", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2442,6 +2444,7 @@ impl AggregateFunction { "GROUPING" => Some(Self::Grouping), "MEDIAN" => Some(Self::Median), "FIRST" => Some(Self::First), + "LAST" => Some(Self::Last), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 297ca6ec550d..fe801269a568 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -521,6 +521,7 @@ impl From for AggregateFunction { protobuf::AggregateFunction::Grouping => Self::Grouping, protobuf::AggregateFunction::Median => Self::Median, protobuf::AggregateFunction::First => Self::First, + protobuf::AggregateFunction::Last => Self::Last, } } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index d849e6f90769..3fe94bd28b62 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -383,6 +383,7 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::Grouping => Self::Grouping, AggregateFunction::Median => Self::Median, AggregateFunction::First => Self::First, + AggregateFunction::Last => Self::Last, } } } @@ -658,6 +659,7 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, AggregateFunction::Median => protobuf::AggregateFunction::Median, AggregateFunction::First => protobuf::AggregateFunction::First, + AggregateFunction::Last => protobuf::AggregateFunction::Last, }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index d5cf4583aebc..1cffdebdefe9 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -34,7 +34,9 @@ use datafusion_expr::{ col, expr, lit, AggregateFunction, Between, BinaryExpr, BuiltinScalarFunction, Cast, Expr, ExprSchemable, GetIndexedField, Like, Operator, TryCast, }; -use sqlparser::ast::{ArrayAgg, Expr as SQLExpr, FirstAgg, TrimWhereField, Value}; +use sqlparser::ast::{ + ArrayAgg, Expr as SQLExpr, FirstAgg, LastAgg, TrimWhereField, Value, +}; use sqlparser::parser::ParserError::ParserError; impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -296,7 +298,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), SQLExpr::FIRST(first_agg) => self.parse_first_agg(first_agg, schema, planner_context), - SQLExpr::LAST(_last_agg) => todo!(), + SQLExpr::LAST(last_agg) => self.parse_last_agg(last_agg, schema, planner_context), _ => Err(DataFusionError::NotImplemented(format!( "Unsupported ast node in sqltorel: {sql:?}" @@ -354,7 +356,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { fn parse_first_agg( &self, - array_agg: FirstAgg, + first_agg: FirstAgg, input_schema: &DFSchema, planner_context: &mut PlannerContext, ) -> Result { @@ -365,7 +367,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { order_by, limit, within_group, - } = array_agg; + } = first_agg; let order_by = if let Some(order_by) = order_by { // TODO: Once sqlparser supports multiple order by clause, handle it @@ -400,6 +402,54 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ))) } + fn parse_last_agg( + &self, + last_agg: LastAgg, + input_schema: &DFSchema, + planner_context: &mut PlannerContext, + ) -> Result { + // Some dialects have special syntax for array_agg. DataFusion only supports it like a function. + let LastAgg { + distinct, + expr, + order_by, + limit, + within_group, + } = last_agg; + + let order_by = if let Some(order_by) = order_by { + // 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 + }; + + if let Some(limit) = limit { + return Err(DataFusionError::NotImplemented(format!( + "LIMIT not supported in LAST: {limit}" + ))); + } + + if within_group { + return Err(DataFusionError::NotImplemented( + "WITHIN GROUP not supported in LAST".to_string(), + )); + } + + let args = + vec![self.sql_expr_to_logical_expr(*expr, input_schema, planner_context)?]; + + // next, aggregate built-ins + let fun = AggregateFunction::Last; + Ok(Expr::AggregateFunction(expr::AggregateFunction::new( + fun, args, distinct, None, order_by, + ))) + } + fn sql_in_list_to_expr( &self, expr: SQLExpr, From 670a327f6195d60f06cec8c8f7b7950f371984a2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 10 May 2023 12:50:55 +0300 Subject: [PATCH 21/53] Update cargo.lock --- datafusion-cli/Cargo.lock | 70 +++++++++++++++++++-------------------- 1 file changed, 34 insertions(+), 36 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 9408547607a7..421dd69e19c3 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -685,9 +685,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.12.1" +version = "3.12.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9b1ce199063694f33ffb7dd4e0ee620741495c32833cde5aa08f02a0bf96f0c8" +checksum = "3c6ed94e98ecff0c12dd1b04c15ec0d7d9458ca8fe806cea6f12954efe74c63b" [[package]] name = "byteorder" @@ -1753,9 +1753,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.61" +version = "0.3.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "445dde2150c55e483f3d8416706b97ec8e8237c307e5b7b4b8dd15e6af2a0730" +checksum = "68c16e1bfd491478ab155fd8b4896b86f9ede344949b641e61501e07c2b8b4d5" dependencies = [ "wasm-bindgen", ] @@ -1832,9 +1832,9 @@ dependencies = [ [[package]] name = "libc" -version = "0.2.142" +version = "0.2.144" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6a987beff54b60ffa6d51982e1aa1146bc42f19bd26be28b0586f252fccf5317" +checksum = "2b00cc1c228a6782d0f076e7b232802e0c5689d41bb5df366f2a6b6621cfdfe1" [[package]] name = "libm" @@ -2361,9 +2361,9 @@ dependencies = [ [[package]] name = "quote" -version = "1.0.26" +version = "1.0.27" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4424af4bf778aae2051a77b60283332f386554255d722233d09fbfc7e30da2fc" +checksum = "8f4f29d145265ec1c483c7c654450edde0bfe043d3938d6972630663356d9500" dependencies = [ "proc-macro2", ] @@ -2804,8 +2804,7 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "sqlparser" version = "0.33.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "355dc4d4b6207ca8a3434fc587db0a8016130a574dbcdbfb93d7f7b5bc5b211a" +source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=870b3de#870b3de433e8c4c3196b3e4984d1cb101288c5c2" dependencies = [ "log", "sqlparser_derive", @@ -2814,8 +2813,7 @@ dependencies = [ [[package]] name = "sqlparser_derive" version = "0.1.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "55fe75cb4a364c7f7ae06c7dbbc8d84bddd85d6cdf9975963c3935bc1991761e" +source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=870b3de#870b3de433e8c4c3196b3e4984d1cb101288c5c2" dependencies = [ "proc-macro2", "quote", @@ -2948,9 +2946,9 @@ dependencies = [ [[package]] name = "time" -version = "0.3.20" +version = "0.3.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "cd0cbfecb4d19b5ea75bb31ad904eb5b9fa13f21079c3b92017ebdf4999a5890" +checksum = "8f3403384eaacbca9923fa06940178ac13e4edb725486d70e8e15881d0c836cc" dependencies = [ "serde", "time-core", @@ -2959,15 +2957,15 @@ dependencies = [ [[package]] name = "time-core" -version = "0.1.0" +version = "0.1.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2e153e1f1acaef8acc537e68b44906d2db6436e2b35ac2c6b42640fff91f00fd" +checksum = "7300fbefb4dadc1af235a9cef3737cea692a9d97e1b9cbcd4ebdae6f8868e6fb" [[package]] name = "time-macros" -version = "0.2.8" +version = "0.2.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fd80a657e71da814b8e5d60d3374fc6d35045062245d80224748ae522dd76f36" +checksum = "372950940a5f07bf38dbe211d7283c9e6d7327df53794992d293e534c733d09b" dependencies = [ "time-core", ] @@ -2998,9 +2996,9 @@ checksum = "1f3ccbac311fea05f86f61904b462b55fb3df8837a366dfc601a0161d0532f20" [[package]] name = "tokio" -version = "1.28.0" +version = "1.28.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3c786bf8134e5a3a166db9b29ab8f48134739014a3eca7bc6bfa95d673b136f" +checksum = "0aa32867d44e6f2ce3385e89dceb990188b8bb0fb25b0cf576647a6f98ac5105" dependencies = [ "autocfg", "bytes", @@ -3255,9 +3253,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31f8dcbc21f30d9b8f2ea926ecb58f6b91192c17e9d33594b3df58b2007ca53b" +checksum = "5b6cb788c4e39112fbe1822277ef6fb3c55cd86b95cb3d3c4c1c9597e4ac74b4" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3265,24 +3263,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "95ce90fd5bcc06af55a641a86428ee4229e44e07033963a2290a8e241607ccb9" +checksum = "35e522ed4105a9d626d885b35d62501b30d9666283a5c8be12c14a8bdafe7822" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.15", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.34" +version = "0.4.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f219e0d211ba40266969f6dbdd90636da12f75bee4fc9d6c23d1260dadb51454" +checksum = "083abe15c5d88556b77bdf7aef403625be9e327ad37c62c4e4129af740168163" dependencies = [ "cfg-if", "js-sys", @@ -3292,9 +3290,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4c21f77c0bedc37fd5dc21f897894a5ca01e7bb159884559461862ae90c0b4c5" +checksum = "358a79a0cb89d21db8120cbfb91392335913e4890665b1a7981d9e956903b434" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3302,22 +3300,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2aff81306fcac3c7515ad4e177f521b5c9a15f2b08f4e32d823066102f35a5f6" +checksum = "4783ce29f09b9d93134d41297aded3a712b7b979e9c6f28c32cb88c973a94869" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.15", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.84" +version = "0.2.85" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0046fef7e28c3804e5e38bfa31ea2a0f73905319b677e57ebe37e49358989b5d" +checksum = "a901d592cafaa4d711bc324edfaff879ac700b19c3dfd60058d2b445be2691eb" [[package]] name = "wasm-streams" @@ -3334,9 +3332,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.61" +version = "0.3.62" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e33b99f4b23ba3eec1a53ac264e35a755f00e966e0065077d6027c0f575b0b97" +checksum = "16b5f940c7edfdc6d12126d98c9ef4d1b3d470011c47c76a6581df47ad9ba721" dependencies = [ "js-sys", "wasm-bindgen", From 4c27b851e90f8b1df66f911ee87df878b87f97e1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 10 May 2023 13:26:29 +0300 Subject: [PATCH 22/53] Remove distinct, and limit from First and last aggregate. --- datafusion/common/Cargo.toml | 4 ++-- datafusion/core/Cargo.toml | 4 +++- datafusion/expr/Cargo.toml | 4 ++-- datafusion/sql/Cargo.toml | 4 ++-- datafusion/sql/src/expr/mod.rs | 20 ++------------------ 5 files changed, 11 insertions(+), 25 deletions(-) diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index cfcd8c14fd0c..f1943d187916 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -49,7 +49,7 @@ parquet = { workspace = true, optional = true } pyo3 = { version = "0.18.0", optional = true } # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } [dev-dependencies] rand = "0.8.4" diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 31a839cd2440..9e154e0286e9 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -86,7 +86,9 @@ pin-project-lite = "^0.2.7" rand = "0.8" smallvec = { version = "1.6", features = ["union"] } # sqlparser = { version = "0.33", features = ["visitor"] } -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"] } +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support", features = ["visitor"] } +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"] } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } tempfile = "3" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } tokio-stream = "0.1" diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index 896956d76629..312784485974 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -40,8 +40,8 @@ arrow = { workspace = true } datafusion-common = { path = "../common", version = "23.0.0" } # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } [dev-dependencies] ctor = "0.2.0" env_logger = "0.10" diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index 9d1d507f1019..bbee3a98a4a7 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -44,8 +44,8 @@ datafusion-expr = { path = "../expr", version = "23.0.0" } log = "^0.4" # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "00719e3"} +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } [dev-dependencies] ctor = "0.2.0" env_logger = "0.10" diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 1cffdebdefe9..e7d59572441f 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -362,10 +362,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { // Some dialects have special syntax for array_agg. DataFusion only supports it like a function. let FirstAgg { - distinct, expr, order_by, - limit, within_group, } = first_agg; @@ -380,12 +378,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { None }; - if let Some(limit) = limit { - return Err(DataFusionError::NotImplemented(format!( - "LIMIT not supported in FIRST: {limit}" - ))); - } - if within_group { return Err(DataFusionError::NotImplemented( "WITHIN GROUP not supported in FIRST".to_string(), @@ -398,7 +390,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins let fun = AggregateFunction::First; Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, order_by, + fun, args, false, None, order_by, ))) } @@ -410,10 +402,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ) -> Result { // Some dialects have special syntax for array_agg. DataFusion only supports it like a function. let LastAgg { - distinct, expr, order_by, - limit, within_group, } = last_agg; @@ -428,12 +418,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { None }; - if let Some(limit) = limit { - return Err(DataFusionError::NotImplemented(format!( - "LIMIT not supported in LAST: {limit}" - ))); - } - if within_group { return Err(DataFusionError::NotImplemented( "WITHIN GROUP not supported in LAST".to_string(), @@ -446,7 +430,7 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins let fun = AggregateFunction::Last; Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, order_by, + fun, args, false, None, order_by, ))) } From e9535bd21e2f093d65740c6a46e81819e5fc675a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 10 May 2023 13:39:39 +0300 Subject: [PATCH 23/53] Add reverse for First and Last Aggregator --- .../physical-expr/src/aggregate/first_last.rs | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index fa4568d84a50..54e491c5e11a 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -85,7 +85,13 @@ impl AggregateExpr for FirstAgg { true } - // TODO: Add support for reverse expr + fn reverse_expr(&self) -> Option> { + Some(Arc::new(LastAgg::new( + self.expr.clone(), + self.name.clone(), + self.data_type.clone(), + ))) + } fn create_sliding_accumulator(&self) -> Result> { Ok(Box::new(FirstAccumulator::try_new(&self.data_type)?)) @@ -143,7 +149,7 @@ impl Accumulator for FirstAccumulator { } fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - // FIRST(first1, first2, first3, ...) = sum1 + sum2 + sum3 + ... + // FIRST(first1, first2, first3, ...) self.update_batch(states) } @@ -218,7 +224,13 @@ impl AggregateExpr for LastAgg { true } - // TODO: Add support for reverse expr + fn reverse_expr(&self) -> Option> { + Some(Arc::new(FirstAgg::new( + self.expr.clone(), + self.name.clone(), + self.data_type.clone(), + ))) + } fn create_sliding_accumulator(&self) -> Result> { Ok(Box::new(LastAccumulator::try_new(&self.data_type)?)) @@ -245,7 +257,7 @@ struct LastAccumulator { } impl LastAccumulator { - /// new First accumulator + /// new Last accumulator pub fn try_new(data_type: &DataType) -> Result { Ok(Self { last: ScalarValue::try_from(data_type)?, @@ -277,7 +289,7 @@ impl Accumulator for LastAccumulator { } fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - // FIRST(first1, first2, first3, ...) = sum1 + sum2 + sum3 + ... + // LAST(last1, last2, last3, ...) self.update_batch(states) } From 2143b723b18b3acbb939afe22e1c9899a919d126 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 10 May 2023 14:29:46 +0300 Subject: [PATCH 24/53] Update cargo lock --- datafusion-cli/Cargo.lock | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 421dd69e19c3..d12a8cac248a 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -2804,7 +2804,7 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "sqlparser" version = "0.33.0" -source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=870b3de#870b3de433e8c4c3196b3e4984d1cb101288c5c2" +source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=1c4c3b8#1c4c3b8e7380d640c3271a61c12de171f87c29bb" dependencies = [ "log", "sqlparser_derive", @@ -2813,7 +2813,7 @@ dependencies = [ [[package]] name = "sqlparser_derive" version = "0.1.1" -source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=870b3de#870b3de433e8c4c3196b3e4984d1cb101288c5c2" +source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=1c4c3b8#1c4c3b8e7380d640c3271a61c12de171f87c29bb" dependencies = [ "proc-macro2", "quote", From c59ce9bb96f4e6fc14ff55de925ab8d9c21cc603 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 10 May 2023 20:59:23 -0500 Subject: [PATCH 25/53] 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 26/53] 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 27/53] 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 a4eced0e98a5c1674005847ca354482f1418d754 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 12 May 2023 11:40:52 +0300 Subject: [PATCH 28/53] Fix projection pushdown bug --- .../sqllogictests/test_files/groupby.slt | 27 +++++++++---------- datafusion/expr/src/expr.rs | 4 +++ datafusion/expr/src/tree_node/expr.rs | 16 ++++++++++- datafusion/expr/src/utils.rs | 7 ++++- 4 files changed, 37 insertions(+), 17 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 2c8689162fe6..40927f8fea66 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2008,9 +2008,6 @@ SELECT a, d, 1 4 913 1 2 848 -statement ok -set datafusion.optimizer.skip_failed_rules = true; - # test_source_sorted_groupby3 query TT @@ -2019,13 +2016,13 @@ EXPLAIN SELECT a, b, FIRST(c ORDER BY a DESC) as first_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST(annotated_data_infinite2.c) ORDER BY [a DESC NULLS FIRST] AS first_c - Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST(annotated_data_infinite2.c) ORDER BY [a DESC NULLS FIRST]]] - TableScan: annotated_data_infinite2 +Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c + Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] + TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST(annotated_data_infinite2.c) ORDER BY [a DESC NULLS FIRST]@2 as first_c] - AggregateExec: mode=Single, gby=[a@1 as a, b@2 as b], aggr=[FIRST(annotated_data_infinite2.c)], ordering_mode=FullyOrdered - CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] + AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST(annotated_data_infinite2.c)], ordering_mode=FullyOrdered + CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III SELECT a, b, FIRST(c ORDER BY a DESC) as first_c @@ -2045,13 +2042,13 @@ EXPLAIN SELECT a, b, LAST(c ORDER BY a DESC) as last_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST(annotated_data_infinite2.c) ORDER BY [a DESC NULLS FIRST] AS last_c - Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST(annotated_data_infinite2.c) ORDER BY [a DESC NULLS FIRST]]] - TableScan: annotated_data_infinite2 +Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c + Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] + TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST(annotated_data_infinite2.c) ORDER BY [a DESC NULLS FIRST]@2 as last_c] - AggregateExec: mode=Single, gby=[a@1 as a, b@2 as b], aggr=[LAST(annotated_data_infinite2.c)], ordering_mode=FullyOrdered - CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], infinite_source=true, output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +ProjectionExec: expr=[a@0 as a, b@1 as b, LAST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] + AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST(annotated_data_infinite2.c)], ordering_mode=FullyOrdered + CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III SELECT a, b, LAST(c ORDER BY a DESC) as last_c 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..112dcf614da1 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( @@ -387,6 +387,20 @@ where .transpose() } +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 diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 1d7aa536f910..1ce2cabf5b87 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -268,7 +268,12 @@ pub fn expr_to_columns(expr: &Expr, accum: &mut HashSet) -> Result<()> { inspect_expr_pre(expr, |expr| { match expr { Expr::Column(qc) => { - accum.insert(qc.clone()); + // Insert if not already inserted + if !accum.contains(qc) { + accum.insert(qc.clone()) + } else { + false + }; } // Use explicit pattern match instead of a default // implementation, so that in the future if someone adds From 344e184da2c30cedb2f29f8dc18daf409aeeed54 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 12 May 2023 12:01:18 +0300 Subject: [PATCH 29/53] 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 9d01d4d4add3a7d190aa7266d0d9836a24581482 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 12 May 2023 14:06:46 +0300 Subject: [PATCH 30/53] combine first_agg and last_agg parsers --- datafusion/sql/src/expr/mod.rs | 78 ++++++++++++---------------------- 1 file changed, 28 insertions(+), 50 deletions(-) diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 8c2d2faa3dc7..b69d7e1e6e84 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -301,8 +301,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Subquery(subquery) => self.parse_scalar_subquery(*subquery, schema, planner_context), SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), - SQLExpr::FIRST(first_agg) => self.parse_first_agg(first_agg, schema, planner_context), - SQLExpr::LAST(last_agg) => self.parse_last_agg(last_agg, schema, planner_context), + SQLExpr::FIRST(first_agg) => self.parse_first_last_agg(FirstLastAgg::First( first_agg), schema, planner_context), + SQLExpr::LAST(last_agg) => self.parse_first_last_agg(FirstLastAgg::Last(last_agg), schema, planner_context), _ => Err(DataFusionError::NotImplemented(format!( "Unsupported ast node in sqltorel: {sql:?}" @@ -358,57 +358,31 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ))) } - fn parse_first_agg( + fn parse_first_last_agg( &self, - first_agg: FirstAgg, + first_last_agg: FirstLastAgg, input_schema: &DFSchema, planner_context: &mut PlannerContext, ) -> Result { - let FirstAgg { - expr, - order_by, - within_group, - } = first_agg; - - let order_by = if let Some(order_by) = order_by { - // 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 + let (expr, order_by, within_group, fun) = match first_last_agg { + FirstLastAgg::First(first_agg) => { + let FirstAgg { + expr, + order_by, + within_group, + } = first_agg; + (expr, order_by, within_group, AggregateFunction::First) + } + FirstLastAgg::Last(last_agg) => { + let LastAgg { + expr, + order_by, + within_group, + } = last_agg; + (expr, order_by, within_group, AggregateFunction::Last) + } }; - if within_group { - return Err(DataFusionError::NotImplemented( - "WITHIN GROUP not supported in FIRST".to_string(), - )); - } - - let args = - vec![self.sql_expr_to_logical_expr(*expr, input_schema, planner_context)?]; - - // next, aggregate built-ins - let fun = AggregateFunction::First; - Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, false, None, order_by, - ))) - } - - fn parse_last_agg( - &self, - last_agg: LastAgg, - input_schema: &DFSchema, - planner_context: &mut PlannerContext, - ) -> Result { - let LastAgg { - expr, - order_by, - within_group, - } = last_agg; - let order_by = if let Some(order_by) = order_by { // TODO: Once sqlparser supports multiple order by clause, handle it Some(vec![self.order_by_to_sort_expr( @@ -422,15 +396,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { if within_group { return Err(DataFusionError::NotImplemented( - "WITHIN GROUP not supported in LAST".to_string(), + "WITHIN GROUP not supported in FIRST/LAST".to_string(), )); } let args = vec![self.sql_expr_to_logical_expr(*expr, input_schema, planner_context)?]; - // next, aggregate built-ins - let fun = AggregateFunction::Last; Ok(Expr::AggregateFunction(expr::AggregateFunction::new( fun, args, false, None, order_by, ))) @@ -607,6 +579,12 @@ fn rewrite_placeholder(expr: &mut Expr, other: &Expr, schema: &DFSchema) -> Resu Ok(()) } +// Wrapper for FirstAgg, and LastAgg function. +enum FirstLastAgg { + First(FirstAgg), + Last(LastAgg), +} + /// Find all [`Expr::Placeholder`] tokens in a logical plan, and try /// to infer their [`DataType`] from the context of their use. fn infer_placeholder_types(expr: Expr, schema: &DFSchema) -> Result { From 8fc5b8209683a9ec53cf0413291b08093e2f27a2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 16 May 2023 18:25:51 +0300 Subject: [PATCH 31/53] Update documentation --- .../user-guide/sql/aggregate_functions.md | 28 +++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index d02c733efc3a..8ae3e5e6f6e2 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -36,6 +36,8 @@ Aggregate functions operate on a set of values to compute a single result. - [min](#min) - [sum](#sum) - [array_agg](#array_agg) +- [first](#first) +- [last](#last) ### `avg` @@ -202,6 +204,32 @@ array_agg(expression [ORDER BY expression]) #### Arguments +- **expression**: Expression to operate on. + Can be a constant, column, or function, and any combination of arithmetic operators. + +### `first` + +Returns the first entry in aggregation group. If ordering requirement is given, returns the first element of the ordered group. + +``` +first(expression [ORDER BY expression]) +``` + +#### Arguments + +- **expression**: Expression to operate on. + Can be a constant, column, or function, and any combination of arithmetic operators. + +### `last` + +Returns the last entry in aggregation group. If ordering requirement is given, returns the last element of the ordered group. + +``` +last(expression [ORDER BY expression]) +``` + +#### Arguments + - **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. From 2ac73d6c9dcd58427e98b17da93a0133d26ca4fc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 16 May 2023 19:22:58 +0300 Subject: [PATCH 32/53] Update subproject --- testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing b/testing index 47f7b56b2568..e81d0c6de359 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 47f7b56b25683202c1fd957668e13f2abafc0f12 +Subproject commit e81d0c6de35948b3be7984af8e00413b314cde6e From dbe08265ad65df383e60d18c396a0a4553942a1f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 13:30:52 +0300 Subject: [PATCH 33/53] initial commit --- datafusion/common/Cargo.toml | 3 +- datafusion/core/Cargo.toml | 3 +- datafusion/expr/Cargo.toml | 3 +- datafusion/sql/Cargo.toml | 3 +- datafusion/sql/src/expr/mod.rs | 64 ++-------------------------------- 5 files changed, 11 insertions(+), 65 deletions(-) diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index f1943d187916..2d6a954e1567 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -50,6 +50,7 @@ pyo3 = { version = "0.18.0", optional = true } # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } [dev-dependencies] rand = "0.8.4" diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 555ad58c1f2e..46d19a97f205 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -88,7 +88,8 @@ smallvec = { version = "1.6", features = ["union"] } # sqlparser = { version = "0.33", features = ["visitor"] } # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support", features = ["visitor"] } # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"] } -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } tempfile = "3" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } tokio-stream = "0.1" diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index ddac1510e204..d63f216d5ea4 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -41,7 +41,8 @@ datafusion-common = { path = "../common", version = "24.0.0" } # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } [dev-dependencies] ctor = "0.2.0" env_logger = "0.10" diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index 5784465f4c9c..9518787a7b51 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -45,7 +45,8 @@ log = "^0.4" # sqlparser = "0.33" # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} # sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } +sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } [dev-dependencies] ctor = "0.2.0" env_logger = "0.10" diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 6e7009288ed5..9394000be24c 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -36,9 +36,7 @@ use datafusion_expr::{ col, expr, lit, AggregateFunction, Between, BinaryExpr, BuiltinScalarFunction, Cast, Expr, ExprSchemable, GetIndexedField, Like, Operator, TryCast, }; -use sqlparser::ast::{ - ArrayAgg, Expr as SQLExpr, FirstAgg, LastAgg, TrimWhereField, Value, -}; +use sqlparser::ast::{ArrayAgg, Expr as SQLExpr, TrimWhereField, Value}; use sqlparser::parser::ParserError::ParserError; impl<'a, S: ContextProvider> SqlToRel<'a, S> { @@ -301,8 +299,8 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Subquery(subquery) => self.parse_scalar_subquery(*subquery, schema, planner_context), SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), - SQLExpr::FIRST(first_agg) => self.parse_first_last_agg(FirstLastAgg::First( first_agg), schema, planner_context), - SQLExpr::LAST(last_agg) => self.parse_first_last_agg(FirstLastAgg::Last(last_agg), schema, planner_context), + // SQLExpr::FIRST(first_agg) => self.parse_first_last_agg(FirstLastAgg::First( first_agg), schema, planner_context), + // SQLExpr::LAST(last_agg) => self.parse_first_last_agg(FirstLastAgg::Last(last_agg), schema, planner_context), _ => Err(DataFusionError::NotImplemented(format!( "Unsupported ast node in sqltorel: {sql:?}" @@ -359,56 +357,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { ))) } - fn parse_first_last_agg( - &self, - first_last_agg: FirstLastAgg, - input_schema: &DFSchema, - planner_context: &mut PlannerContext, - ) -> Result { - let (expr, order_by, within_group, fun) = match first_last_agg { - FirstLastAgg::First(first_agg) => { - let FirstAgg { - expr, - order_by, - within_group, - } = first_agg; - (expr, order_by, within_group, AggregateFunction::First) - } - FirstLastAgg::Last(last_agg) => { - let LastAgg { - expr, - order_by, - within_group, - } = last_agg; - (expr, order_by, within_group, AggregateFunction::Last) - } - }; - - let order_by = if let Some(order_by) = order_by { - // 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 - }; - - if within_group { - return Err(DataFusionError::NotImplemented( - "WITHIN GROUP not supported in FIRST/LAST".to_string(), - )); - } - - let args = - vec![self.sql_expr_to_logical_expr(*expr, input_schema, planner_context)?]; - - Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, false, None, order_by, - ))) - } - fn sql_in_list_to_expr( &self, expr: SQLExpr, @@ -580,12 +528,6 @@ fn rewrite_placeholder(expr: &mut Expr, other: &Expr, schema: &DFSchema) -> Resu Ok(()) } -// Wrapper for FirstAgg, and LastAgg function. -enum FirstLastAgg { - First(FirstAgg), - Last(LastAgg), -} - /// Find all [`Expr::Placeholder`] tokens in a logical plan, and try /// to infer their [`DataType`] from the context of their use. fn infer_placeholder_types(expr: Expr, schema: &DFSchema) -> Result { From 118fa92667abbf120ce8f3ace8de68460525a024 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 13:47:55 +0300 Subject: [PATCH 34/53] Add test code --- .../physical_optimizer/sort_enforcement.rs | 64 +++++++++++++++++++ datafusion/expr/src/aggregate_function.rs | 12 ++-- .../expr/src/type_coercion/aggregates.rs | 4 +- .../physical-expr/src/aggregate/build_in.rs | 4 +- .../proto/src/logical_plan/from_proto.rs | 4 +- datafusion/proto/src/logical_plan/to_proto.rs | 8 +-- datafusion/sql/src/expr/function.rs | 5 +- 7 files changed, 85 insertions(+), 16 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index f71c79e9fc82..c12df4ac1389 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2894,3 +2894,67 @@ mod tests { ) } } + +mod tmp_tests{ + use tempfile::TempDir; + use datafusion_common::Result; + use datafusion_execution::config::SessionConfig; + use crate::assert_batches_eq; + use crate::physical_plan::{collect, displayable}; + use crate::prelude::SessionContext; + + #[tokio::test] + async fn test_first_value() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(1); + let ctx = SessionContext::with_config(config); + ctx.sql("CREATE EXTERNAL TABLE annotated_data_infinite ( + ts INTEGER, + inc_col INTEGER, + desc_col INTEGER, + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (ts ASC) + LOCATION 'tests/data/window_1.csv'").await?; + + let sql = "SELECT FIRST_VALUE(inc_col ORDER BY ts DESC), ts, inc_col + FROM annotated_data_infinite + ORDER BY ts ASC"; + + 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(); + let expected = { + vec![ + "GlobalLimitExec: skip=0, fetch=5", + " ProjectionExec: expr=[ts@0 as ts, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1]", + " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted]", + ] + }; + + 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 = collect(physical_plan, ctx.task_ctx()).await?; + let expected = vec![ + "+----+-----+", + "| ts | rn1 |", + "+----+-----+", + "| 1 | 1 |", + "| 1 | 2 |", + "| 5 | 3 |", + "| 9 | 4 |", + "| 10 | 5 |", + "+----+-----+", + ]; + assert_batches_eq!(expected, &actual); + Ok(()) + } + +} diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 2a46d5a50baf..0aa2159067ef 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -43,9 +43,9 @@ pub enum AggregateFunction { /// array_agg ArrayAgg, /// first - First, + FirstValue, /// last - Last, + LastValue, /// Variance (Sample) Variance, /// Variance (Population) @@ -105,6 +105,8 @@ impl FromStr for AggregateFunction { "min" => AggregateFunction::Min, "sum" => AggregateFunction::Sum, "array_agg" => AggregateFunction::ArrayAgg, + "first_value" => AggregateFunction::FirstValue, + "last_value" => AggregateFunction::LastValue, // statistical "corr" => AggregateFunction::Correlation, "covar" => AggregateFunction::Covariance, @@ -186,7 +188,7 @@ pub fn return_type( Ok(coerced_data_types[0].clone()) } AggregateFunction::Grouping => Ok(DataType::Int32), - AggregateFunction::First | AggregateFunction::Last => { + AggregateFunction::FirstValue | AggregateFunction::LastValue => { Ok(coerced_data_types[0].clone()) } } @@ -240,8 +242,8 @@ pub fn signature(fun: &AggregateFunction) -> Signature { | AggregateFunction::StddevPop | AggregateFunction::Median | AggregateFunction::ApproxMedian - | AggregateFunction::First - | AggregateFunction::Last => { + | AggregateFunction::FirstValue + | AggregateFunction::LastValue => { Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable) } AggregateFunction::Covariance | AggregateFunction::CovariancePop => { diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 1fe1af9c2f83..4f02bf3dfd2a 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -265,8 +265,8 @@ pub fn coerce_types( Ok(input_types.to_vec()) } AggregateFunction::Median - | AggregateFunction::First - | AggregateFunction::Last => Ok(input_types.to_vec()), + | AggregateFunction::FirstValue + | AggregateFunction::LastValue => Ok(input_types.to_vec()), AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]), } } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 88653205756d..4853b033e577 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -308,12 +308,12 @@ pub fn create_aggregate_expr( "MEDIAN(DISTINCT) aggregations are not available".to_string(), )); } - (AggregateFunction::First, _) => Arc::new(expressions::FirstAgg::new( + (AggregateFunction::FirstValue, _) => Arc::new(expressions::FirstAgg::new( input_phy_exprs[0].clone(), name, input_phy_types[0].clone(), )), - (AggregateFunction::Last, _) => Arc::new(expressions::LastAgg::new( + (AggregateFunction::LastValue, _) => Arc::new(expressions::LastAgg::new( input_phy_exprs[0].clone(), name, input_phy_types[0].clone(), diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index c182aa38747f..762a73bd4919 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -529,8 +529,8 @@ impl From for AggregateFunction { protobuf::AggregateFunction::ApproxMedian => Self::ApproxMedian, protobuf::AggregateFunction::Grouping => Self::Grouping, protobuf::AggregateFunction::Median => Self::Median, - protobuf::AggregateFunction::First => Self::First, - protobuf::AggregateFunction::Last => Self::Last, + protobuf::AggregateFunction::First => Self::FirstValue, + protobuf::AggregateFunction::Last => Self::LastValue, } } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index b5f0bead5fb7..6f9120a4a7c0 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -388,8 +388,8 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::ApproxMedian => Self::ApproxMedian, AggregateFunction::Grouping => Self::Grouping, AggregateFunction::Median => Self::Median, - AggregateFunction::First => Self::First, - AggregateFunction::Last => Self::Last, + AggregateFunction::FirstValue => Self::First, + AggregateFunction::LastValue => Self::Last, } } } @@ -669,8 +669,8 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, AggregateFunction::Median => protobuf::AggregateFunction::Median, - AggregateFunction::First => protobuf::AggregateFunction::First, - AggregateFunction::Last => protobuf::AggregateFunction::Last, + AggregateFunction::FirstValue => protobuf::AggregateFunction::First, + AggregateFunction::LastValue => protobuf::AggregateFunction::Last, }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 2bc3125d733d..3d1c5b1034e6 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -110,10 +110,13 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins if let Ok(fun) = AggregateFunction::from_str(&name) { let distinct = function.distinct; + let order_by = function.order_by.into_iter().map(|e| self.order_by_to_sort_expr(e, schema, planner_context)).collect::>>()?; + let order_by = (!order_by.is_empty()).then_some(order_by); + println!("order by:{:?}", order_by); let (fun, args) = self.aggregate_fn_to_expr(fun, function.args, schema, planner_context)?; return Ok(Expr::AggregateFunction(expr::AggregateFunction::new( - fun, args, distinct, None, None, + fun, args, distinct, None, order_by, ))); }; From 228d97318647e9292943ebe8c8a21fedc33a971c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 13:55:51 +0300 Subject: [PATCH 35/53] initial version --- .../physical_optimizer/sort_enforcement.rs | 43 ++++++++----------- .../sqllogictests/test_files/groupby.slt | 6 +-- datafusion/sql/src/expr/function.rs | 6 ++- 3 files changed, 25 insertions(+), 30 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index c12df4ac1389..689e8562f9aa 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2895,20 +2895,20 @@ mod tests { } } -mod tmp_tests{ - use tempfile::TempDir; - use datafusion_common::Result; - use datafusion_execution::config::SessionConfig; +mod tmp_tests { use crate::assert_batches_eq; use crate::physical_plan::{collect, displayable}; use crate::prelude::SessionContext; + use datafusion_common::Result; + use datafusion_execution::config::SessionConfig; + use tempfile::TempDir; #[tokio::test] async fn test_first_value() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(1); + let config = SessionConfig::new().with_target_partitions(1); let ctx = SessionContext::with_config(config); - ctx.sql("CREATE EXTERNAL TABLE annotated_data_infinite ( + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data_infinite ( ts INTEGER, inc_col INTEGER, desc_col INTEGER, @@ -2916,11 +2916,14 @@ mod tmp_tests{ STORED AS CSV WITH HEADER ROW WITH ORDER (ts ASC) - LOCATION 'tests/data/window_1.csv'").await?; + LOCATION 'tests/data/window_1.csv'", + ) + .await?; - let sql = "SELECT FIRST_VALUE(inc_col ORDER BY ts DESC), ts, inc_col - FROM annotated_data_infinite - ORDER BY ts ASC"; + // let sql = "SELECT FIRST_VALUE(inc_col ORDER BY ts ASC) as first + // FROM annotated_data_infinite"; + let sql = "SELECT LAST_VALUE(inc_col ORDER BY ts ASC) as last + FROM annotated_data_infinite"; let msg = format!("Creating logical plan for '{sql}'"); let dataframe = ctx.sql(sql).await.expect(&msg); @@ -2928,9 +2931,8 @@ mod tmp_tests{ let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let expected = { vec![ - "GlobalLimitExec: skip=0, fetch=5", - " ProjectionExec: expr=[ts@0 as ts, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1]", - " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }], mode=[Sorted]", + "ProjectionExec: expr=[LASTVALUE(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST]@0 as last]", + " AggregateExec: mode=Single, gby=[], aggr=[LASTVALUE(annotated_data_infinite.inc_col)]", ] }; @@ -2942,19 +2944,8 @@ mod tmp_tests{ "\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" ); let actual = collect(physical_plan, ctx.task_ctx()).await?; - let expected = vec![ - "+----+-----+", - "| ts | rn1 |", - "+----+-----+", - "| 1 | 1 |", - "| 1 | 2 |", - "| 5 | 3 |", - "| 9 | 4 |", - "| 10 | 5 |", - "+----+-----+", - ]; + let expected = vec!["+------+", "| last |", "+------+", "| 305 |", "+------+"]; assert_batches_eq!(expected, &actual); Ok(()) } - } diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 11ce419c551d..d495562d5b4b 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2011,7 +2011,7 @@ SELECT a, d, # test_source_sorted_groupby3 query TT -EXPLAIN SELECT a, b, FIRST(c ORDER BY a DESC) as first_c +EXPLAIN SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c FROM annotated_data_infinite2 GROUP BY a, b ---- @@ -2037,7 +2037,7 @@ SELECT a, b, FIRST(c ORDER BY a DESC) as first_c # test_source_sorted_groupby4 query TT -EXPLAIN SELECT a, b, LAST(c ORDER BY a DESC) as last_c +EXPLAIN SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c FROM annotated_data_infinite2 GROUP BY a, b ---- @@ -2051,7 +2051,7 @@ ProjectionExec: expr=[a@0 as a, b@1 as b, LAST(annotated_data_infinite2.c) ORDER ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III -SELECT a, b, LAST(c ORDER BY a DESC) as last_c +SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c FROM annotated_data_infinite2 GROUP BY a, b ---- diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 3d1c5b1034e6..4112d22bc17c 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -110,7 +110,11 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { // next, aggregate built-ins if let Ok(fun) = AggregateFunction::from_str(&name) { let distinct = function.distinct; - let order_by = function.order_by.into_iter().map(|e| self.order_by_to_sort_expr(e, schema, planner_context)).collect::>>()?; + let order_by = function + .order_by + .into_iter() + .map(|e| self.order_by_to_sort_expr(e, schema, planner_context)) + .collect::>>()?; let order_by = (!order_by.is_empty()).then_some(order_by); println!("order by:{:?}", order_by); let (fun, args) = From c9cca163ed67458ad16aa16dcf9f5e13fcc41178 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 15:18:09 +0300 Subject: [PATCH 36/53] simplify prints --- .../physical_optimizer/sort_enforcement.rs | 5 ++-- datafusion/core/tests/dataframe_functions.rs | 20 ++++++------- datafusion/core/tests/sql/aggregates.rs | 4 +-- datafusion/expr/src/aggregate_function.rs | 5 +++- datafusion/expr/src/utils.rs | 14 ++++++++++ datafusion/expr/src/window_function.rs | 28 ++++++++++--------- 6 files changed, 47 insertions(+), 29 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 689e8562f9aa..71f603d49064 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2901,7 +2901,6 @@ mod tmp_tests { use crate::prelude::SessionContext; use datafusion_common::Result; use datafusion_execution::config::SessionConfig; - use tempfile::TempDir; #[tokio::test] async fn test_first_value() -> Result<()> { @@ -2931,8 +2930,8 @@ mod tmp_tests { let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let expected = { vec![ - "ProjectionExec: expr=[LASTVALUE(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST]@0 as last]", - " AggregateExec: mode=Single, gby=[], aggr=[LASTVALUE(annotated_data_infinite.inc_col)]", + "ProjectionExec: expr=[LAST_VALUE(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST]@0 as last]", + " AggregateExec: mode=Single, gby=[], aggr=[LAST_VALUE(annotated_data_infinite.inc_col)]", ] }; diff --git a/datafusion/core/tests/dataframe_functions.rs b/datafusion/core/tests/dataframe_functions.rs index 2f4e4d9d8c98..e1173e1d5c07 100644 --- a/datafusion/core/tests/dataframe_functions.rs +++ b/datafusion/core/tests/dataframe_functions.rs @@ -155,11 +155,11 @@ async fn test_fn_approx_median() -> Result<()> { let expr = approx_median(col("b")); let expected = vec![ - "+----------------------+", - "| APPROXMEDIAN(test.b) |", - "+----------------------+", - "| 10 |", - "+----------------------+", + "+-----------------------+", + "| APPROX_MEDIAN(test.b) |", + "+-----------------------+", + "| 10 |", + "+-----------------------+", ]; let df = create_test_table().await?; @@ -175,11 +175,11 @@ async fn test_fn_approx_percentile_cont() -> Result<()> { let expr = approx_percentile_cont(col("b"), lit(0.5)); let expected = vec![ - "+-------------------------------------------+", - "| APPROXPERCENTILECONT(test.b,Float64(0.5)) |", - "+-------------------------------------------+", - "| 10 |", - "+-------------------------------------------+", + "+---------------------------------------------+", + "| APPROX_PERCENTILE_CONT(test.b,Float64(0.5)) |", + "+---------------------------------------------+", + "| 10 |", + "+---------------------------------------------+", ]; let df = create_test_table().await?; diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index e847ea0c0ebf..3ff81581c096 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -29,7 +29,7 @@ async fn csv_query_array_agg_distinct() -> Result<()> { // The results for this query should be something like the following: // +------------------------------------------+ - // | ARRAYAGG(DISTINCT aggregate_test_100.c2) | + // | ARRAY_AGG(DISTINCT aggregate_test_100.c2) | // +------------------------------------------+ // | [4, 2, 3, 5, 1] | // +------------------------------------------+ @@ -37,7 +37,7 @@ async fn csv_query_array_agg_distinct() -> Result<()> { assert_eq!( *actual[0].schema(), Schema::new(vec![Field::new_list( - "ARRAYAGG(DISTINCT aggregate_test_100.c2)", + "ARRAY_AGG(DISTINCT aggregate_test_100.c2)", Field::new("item", DataType::UInt32, true), false ),]) diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 0aa2159067ef..c620f353931a 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -17,6 +17,7 @@ //! Aggregate function module contains all built-in aggregate functions definitions +use crate::utils::convert_camel_uppercase_snake; use crate::{type_coercion::aggregates::*, Signature, TypeSignature, Volatility}; use arrow::datatypes::{DataType, Field}; use datafusion_common::{DataFusionError, Result}; @@ -83,7 +84,9 @@ pub enum AggregateFunction { impl fmt::Display for AggregateFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { // uppercase of the debug. - write!(f, "{}", format!("{self:?}").to_uppercase()) + // Convert Camel form to uppercase snake + // such as FirstValue => FIRST_VALUE + write!(f, "{}", convert_camel_uppercase_snake(format!("{self:?}"))) } } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 1d7aa536f910..6f5a707d5fcd 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -54,6 +54,20 @@ pub fn exprlist_to_columns(expr: &[Expr], accum: &mut HashSet) -> Result Ok(()) } +/// Convert Camel form to uppercase snake +/// such as FirstValue => FIRST_VALUE +pub fn convert_camel_uppercase_snake(in_str: String) -> String { + let mut res = String::new(); + for ch in in_str.chars() { + if ch.is_uppercase() && !res.is_empty() { + res = format!("{}_{}", res, ch.to_uppercase()); + } else { + res = format!("{}{}", res, ch.to_uppercase()); + } + } + res +} + /// Count the number of distinct exprs in a list of group by expressions. If the /// first element is a `GroupingSet` expression then it must be the only expr. pub fn grouping_set_expr_count(group_expr: &[Expr]) -> Result { diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 2d91dca8cc06..a1c816216273 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -23,6 +23,7 @@ use crate::aggregate_function::AggregateFunction; use crate::type_coercion::functions::data_types; +use crate::utils::convert_camel_uppercase_snake; use crate::{aggregate_function, AggregateUDF, Signature, TypeSignature, Volatility}; use arrow::datatypes::DataType; use datafusion_common::{DataFusionError, Result}; @@ -53,19 +54,20 @@ pub fn find_df_window_func(name: &str) -> Option { impl fmt::Display for BuiltInWindowFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - match self { - BuiltInWindowFunction::RowNumber => write!(f, "ROW_NUMBER"), - BuiltInWindowFunction::Rank => write!(f, "RANK"), - BuiltInWindowFunction::DenseRank => write!(f, "DENSE_RANK"), - BuiltInWindowFunction::PercentRank => write!(f, "PERCENT_RANK"), - BuiltInWindowFunction::CumeDist => write!(f, "CUME_DIST"), - BuiltInWindowFunction::Ntile => write!(f, "NTILE"), - BuiltInWindowFunction::Lag => write!(f, "LAG"), - BuiltInWindowFunction::Lead => write!(f, "LEAD"), - BuiltInWindowFunction::FirstValue => write!(f, "FIRST_VALUE"), - BuiltInWindowFunction::LastValue => write!(f, "LAST_VALUE"), - BuiltInWindowFunction::NthValue => write!(f, "NTH_VALUE"), - } + // match self { + // BuiltInWindowFunction::RowNumber => write!(f, "ROW_NUMBER"), + // BuiltInWindowFunction::Rank => write!(f, "RANK"), + // BuiltInWindowFunction::DenseRank => write!(f, "DENSE_RANK"), + // BuiltInWindowFunction::PercentRank => write!(f, "PERCENT_RANK"), + // BuiltInWindowFunction::CumeDist => write!(f, "CUME_DIST"), + // BuiltInWindowFunction::Ntile => write!(f, "NTILE"), + // BuiltInWindowFunction::Lag => write!(f, "LAG"), + // BuiltInWindowFunction::Lead => write!(f, "LEAD"), + // BuiltInWindowFunction::FirstValue => write!(f, "FIRST_VALUE"), + // BuiltInWindowFunction::LastValue => write!(f, "LAST_VALUE"), + // BuiltInWindowFunction::NthValue => write!(f, "NTH_VALUE"), + // } + write!(f, "{}", convert_camel_uppercase_snake(format!("{self:?}"))) } } From 8a87fdba9c7ae4a9a4712069527a5d8862e39eaf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 15:29:16 +0300 Subject: [PATCH 37/53] minor changes --- .../core/src/physical_optimizer/sort_enforcement.rs | 9 ++++++--- datafusion/physical-expr/src/aggregate/first_last.rs | 3 +++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 71f603d49064..dfbca7d22174 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2921,7 +2921,10 @@ mod tmp_tests { // let sql = "SELECT FIRST_VALUE(inc_col ORDER BY ts ASC) as first // FROM annotated_data_infinite"; - let sql = "SELECT LAST_VALUE(inc_col ORDER BY ts ASC) as last + // let sql = "SELECT LAST_VALUE(inc_col ORDER BY ts ASC) as last + // FROM annotated_data_infinite"; + + let sql = "SELECT LAST_VALUE(inc_col) OVER() as last FROM annotated_data_infinite"; let msg = format!("Creating logical plan for '{sql}'"); @@ -2930,8 +2933,8 @@ mod tmp_tests { let formatted = displayable(physical_plan.as_ref()).indent().to_string(); let expected = { vec![ - "ProjectionExec: expr=[LAST_VALUE(annotated_data_infinite.inc_col) ORDER BY [annotated_data_infinite.ts ASC NULLS LAST]@0 as last]", - " AggregateExec: mode=Single, gby=[], aggr=[LAST_VALUE(annotated_data_infinite.inc_col)]", + "ProjectionExec: expr=[LAST_VALUE(annotated_data_infinite.inc_col) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as last]", + " WindowAggExec: wdw=[LAST_VALUE(annotated_data_infinite.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data_infinite.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", ] }; diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 54e491c5e11a..34c6eb568a1c 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -43,6 +43,7 @@ impl FirstAgg { name: impl Into, data_type: DataType, ) -> Self { + println!("first agg init"); Self { name: name.into(), data_type, @@ -182,6 +183,7 @@ impl LastAgg { name: impl Into, data_type: DataType, ) -> Self { + println!("last agg init"); Self { name: name.into(), data_type, @@ -259,6 +261,7 @@ struct LastAccumulator { impl LastAccumulator { /// new Last accumulator pub fn try_new(data_type: &DataType) -> Result { + println!("last accumulator init"); Ok(Self { last: ScalarValue::try_from(data_type)?, count: 0, From 1db5683934de34a3a9a9f692ae44ca520ee7cd38 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 16:04:11 +0300 Subject: [PATCH 38/53] sqllogictests pass --- .../physical_optimizer/sort_enforcement.rs | 4 +- .../sqllogictests/test_files/aggregate.slt | 2 +- .../sqllogictests/test_files/groupby.slt | 58 +++++++++---------- .../tests/sqllogictests/test_files/window.slt | 10 ++-- datafusion/expr/src/window_function.rs | 19 +----- 5 files changed, 40 insertions(+), 53 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index dfbca7d22174..541f04249a58 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2921,8 +2921,8 @@ mod tmp_tests { // let sql = "SELECT FIRST_VALUE(inc_col ORDER BY ts ASC) as first // FROM annotated_data_infinite"; - // let sql = "SELECT LAST_VALUE(inc_col ORDER BY ts ASC) as last - // FROM annotated_data_infinite"; + // let sql = "SELECT LAST_VALUE(inc_col ORDER BY ts ASC) as last + // FROM annotated_data_infinite"; let sql = "SELECT LAST_VALUE(inc_col) OVER() as last FROM annotated_data_infinite"; diff --git a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt index 47d7d031ce78..811b4d1f9d78 100644 --- a/datafusion/core/tests/sqllogictests/test_files/aggregate.slt +++ b/datafusion/core/tests/sqllogictests/test_files/aggregate.slt @@ -41,7 +41,7 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' ####### # https://github.com/apache/arrow-datafusion/issues/3353 -statement error DataFusion error: Schema error: Schema contains duplicate unqualified field name "APPROXDISTINCT\(aggregate_test_100\.c9\)" +statement error DataFusion error: Schema error: Schema contains duplicate unqualified field name "APPROX_DISTINCT\(aggregate_test_100\.c9\)" SELECT approx_distinct(c9) count_c9, approx_distinct(cast(c9 as varchar)) count_c9_str FROM aggregate_test_100 # csv_query_approx_percentile_cont_with_weight diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index d495562d5b4b..09a35d24c7cf 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2016,16 +2016,16 @@ EXPLAIN SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS first_c +--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III -SELECT a, b, FIRST(c ORDER BY a DESC) as first_c +SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c FROM annotated_data_infinite2 GROUP BY a, b ---- @@ -2042,12 +2042,12 @@ EXPLAIN SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c GROUP BY a, b ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c ---Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] +Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS last_c +--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan -ProjectionExec: expr=[a@0 as a, b@1 as b, LAST(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2091,12 +2091,12 @@ EXPLAIN SELECT country, (ARRAY_AGG(amount ORDER BY amount ASC)) AS amounts 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]]] +Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST] AS amounts +--Aggregate: groupBy=[[sales_global.country]], aggr=[[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]]] ----TableScan: sales_global projection=[country, amount] 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)] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2120,13 +2120,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, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(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)] +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2173,14 +2173,14 @@ 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, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(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 +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2209,14 +2209,14 @@ 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, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +--Aggregate: groupBy=[[s.country, s.zip_code]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], SUM(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 +ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] +--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2245,14 +2245,14 @@ 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, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] AS amounts, SUM(s.amount) AS sum1 +--Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST], SUM(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 +ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index 80423a5fe2e9..72a39418846b 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -2007,16 +2007,16 @@ query TT EXPLAIN SELECT ARRAY_AGG(c13) as array_agg1 FROM (SELECT * FROM aggregate_test_100 ORDER BY c13 LIMIT 1) ---- logical_plan -Projection: ARRAYAGG(aggregate_test_100.c13) AS array_agg1 ---Aggregate: groupBy=[[]], aggr=[[ARRAYAGG(aggregate_test_100.c13)]] +Projection: ARRAY_AGG(aggregate_test_100.c13) AS array_agg1 +--Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(aggregate_test_100.c13)]] ----Limit: skip=0, fetch=1 ------Sort: aggregate_test_100.c13 ASC NULLS LAST, fetch=1 --------TableScan: aggregate_test_100 projection=[c13] physical_plan -ProjectionExec: expr=[ARRAYAGG(aggregate_test_100.c13)@0 as array_agg1] ---AggregateExec: mode=Final, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)] +ProjectionExec: expr=[ARRAY_AGG(aggregate_test_100.c13)@0 as array_agg1] +--AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)] +------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(aggregate_test_100.c13)] --------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ----------GlobalLimitExec: skip=0, fetch=1 ------------SortExec: fetch=1, expr=[c13@0 ASC NULLS LAST] diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index a1c816216273..a256f2da1c3e 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -43,10 +43,10 @@ pub enum WindowFunction { /// Find DataFusion's built-in window function by name. pub fn find_df_window_func(name: &str) -> Option { let name = name.to_lowercase(); - if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { - Some(WindowFunction::AggregateFunction(aggregate)) - } else if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { + if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { Some(WindowFunction::BuiltInWindowFunction(built_in_function)) + } else if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { + Some(WindowFunction::AggregateFunction(aggregate)) } else { None } @@ -54,19 +54,6 @@ pub fn find_df_window_func(name: &str) -> Option { impl fmt::Display for BuiltInWindowFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - // match self { - // BuiltInWindowFunction::RowNumber => write!(f, "ROW_NUMBER"), - // BuiltInWindowFunction::Rank => write!(f, "RANK"), - // BuiltInWindowFunction::DenseRank => write!(f, "DENSE_RANK"), - // BuiltInWindowFunction::PercentRank => write!(f, "PERCENT_RANK"), - // BuiltInWindowFunction::CumeDist => write!(f, "CUME_DIST"), - // BuiltInWindowFunction::Ntile => write!(f, "NTILE"), - // BuiltInWindowFunction::Lag => write!(f, "LAG"), - // BuiltInWindowFunction::Lead => write!(f, "LEAD"), - // BuiltInWindowFunction::FirstValue => write!(f, "FIRST_VALUE"), - // BuiltInWindowFunction::LastValue => write!(f, "LAST_VALUE"), - // BuiltInWindowFunction::NthValue => write!(f, "NTH_VALUE"), - // } write!(f, "{}", convert_camel_uppercase_snake(format!("{self:?}"))) } } From 2b2ae1de6d66ffbee5115c8a4f56e48ea94d175b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 16:14:52 +0300 Subject: [PATCH 39/53] All tests pass --- .../physical_optimizer/sort_enforcement.rs | 57 ------------------- .../core/tests/user_defined_aggregates.rs | 8 +-- datafusion/sql/tests/integration_test.rs | 8 +-- 3 files changed, 8 insertions(+), 65 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 541f04249a58..f71c79e9fc82 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -2894,60 +2894,3 @@ mod tests { ) } } - -mod tmp_tests { - use crate::assert_batches_eq; - use crate::physical_plan::{collect, displayable}; - use crate::prelude::SessionContext; - use datafusion_common::Result; - use datafusion_execution::config::SessionConfig; - - #[tokio::test] - async fn test_first_value() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data_infinite ( - ts INTEGER, - inc_col INTEGER, - desc_col INTEGER, - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (ts ASC) - LOCATION 'tests/data/window_1.csv'", - ) - .await?; - - // let sql = "SELECT FIRST_VALUE(inc_col ORDER BY ts ASC) as first - // FROM annotated_data_infinite"; - // let sql = "SELECT LAST_VALUE(inc_col ORDER BY ts ASC) as last - // FROM annotated_data_infinite"; - - let sql = "SELECT LAST_VALUE(inc_col) OVER() as last - FROM annotated_data_infinite"; - - 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(); - let expected = { - vec![ - "ProjectionExec: expr=[LAST_VALUE(annotated_data_infinite.inc_col) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as last]", - " WindowAggExec: wdw=[LAST_VALUE(annotated_data_infinite.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data_infinite.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", - ] - }; - - 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 = collect(physical_plan, ctx.task_ctx()).await?; - let expected = vec!["+------+", "| last |", "+------+", "| 305 |", "+------+"]; - assert_batches_eq!(expected, &actual); - Ok(()) - } -} diff --git a/datafusion/core/tests/user_defined_aggregates.rs b/datafusion/core/tests/user_defined_aggregates.rs index b998fc355fa6..1047f73df4cd 100644 --- a/datafusion/core/tests/user_defined_aggregates.rs +++ b/datafusion/core/tests/user_defined_aggregates.rs @@ -43,10 +43,10 @@ use datafusion_common::cast::as_primitive_array; /// Basic query for with a udaf returning a structure async fn test_udf_returning_struct() { let ctx = udaf_struct_context(); - let sql = "SELECT first_value(value, time) from t"; + let sql = "SELECT first(value, time) from t"; let expected = vec![ "+------------------------------------------------+", - "| first_value(t.value,t.time) |", + "| first(t.value,t.time) |", "+------------------------------------------------+", "| {value: 2.0, time: 1970-01-01T00:00:00.000002} |", "+------------------------------------------------+", @@ -58,7 +58,7 @@ async fn test_udf_returning_struct() { /// Demonstrate extracting the fields from the a structure using a subquery async fn test_udf_returning_struct_sq() { let ctx = udaf_struct_context(); - let sql = "select sq.first['value'], sq.first['time'] from (SELECT first_value(value, time) as first from t) as sq"; + let sql = "select sq.first['value'], sq.first['time'] from (SELECT first(value, time) as first from t) as sq"; let expected = vec![ "+-----------------+----------------------------+", "| sq.first[value] | sq.first[time] |", @@ -117,7 +117,7 @@ fn register_aggregate(ctx: &mut SessionContext) { let volatility = Volatility::Immutable; - let name = "first_value"; + let name = "first"; let first = AggregateUDF::new( name, diff --git a/datafusion/sql/tests/integration_test.rs b/datafusion/sql/tests/integration_test.rs index e1acfcd7c5ca..1b697807f4d4 100644 --- a/datafusion/sql/tests/integration_test.rs +++ b/datafusion/sql/tests/integration_test.rs @@ -1495,8 +1495,8 @@ fn select_count_column() { #[test] fn select_approx_median() { let sql = "SELECT approx_median(age) FROM person"; - let expected = "Projection: APPROXMEDIAN(person.age)\ - \n Aggregate: groupBy=[[]], aggr=[[APPROXMEDIAN(person.age)]]\ + let expected = "Projection: APPROX_MEDIAN(person.age)\ + \n Aggregate: groupBy=[[]], aggr=[[APPROX_MEDIAN(person.age)]]\ \n TableScan: person"; quick_test(sql, expected); } @@ -2427,8 +2427,8 @@ fn approx_median_window() { let sql = "SELECT order_id, APPROX_MEDIAN(qty) OVER(PARTITION BY order_id) from orders"; let expected = "\ - Projection: orders.order_id, APPROXMEDIAN(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ - \n WindowAggr: windowExpr=[[APPROXMEDIAN(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ + Projection: orders.order_id, APPROX_MEDIAN(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\ + \n WindowAggr: windowExpr=[[APPROX_MEDIAN(orders.qty) PARTITION BY [orders.order_id] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]]\ \n TableScan: orders"; quick_test(sql, expected); } From 2804350e0dce78aad890fa2a1fb8f528c2857932 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 16:42:49 +0300 Subject: [PATCH 40/53] update proto function names --- datafusion-cli/Cargo.lock | 241 ++++++++---------- datafusion/proto/proto/datafusion.proto | 6 +- datafusion/proto/src/generated/pbjson.rs | 12 +- datafusion/proto/src/generated/prost.rs | 14 +- .../proto/src/logical_plan/from_proto.rs | 4 +- datafusion/proto/src/logical_plan/to_proto.rs | 12 +- datafusion/sql/src/expr/function.rs | 1 - 7 files changed, 136 insertions(+), 154 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 0d35a0bdb3bc..3bc2b967fb69 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -302,7 +302,7 @@ checksum = "b9ccdd8f2a161be9bd5c023df56f1b2a0bd1d83872ae53b71a84a12c9bf6e842" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -507,7 +507,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls", + "hyper-rustls 0.23.2", "lazy_static", "pin-project-lite", "tokio", @@ -832,16 +832,6 @@ dependencies = [ "winapi", ] -[[package]] -name = "codespan-reporting" -version = "0.11.1" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3538270d33cc669650c4b093848450d380def10c331d38c768e34cac80576e6e" -dependencies = [ - "termcolor", - "unicode-width", -] - [[package]] name = "comfy-table" version = "6.1.4" @@ -952,50 +942,6 @@ dependencies = [ "memchr", ] -[[package]] -name = "cxx" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f61f1b6389c3fe1c316bf8a4dccc90a38208354b330925bce1f74a6c4756eb93" -dependencies = [ - "cc", - "cxxbridge-flags", - "cxxbridge-macro", - "link-cplusplus", -] - -[[package]] -name = "cxx-build" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12cee708e8962df2aeb38f594aae5d827c022b6460ac71a7a3e2c3c2aae5a07b" -dependencies = [ - "cc", - "codespan-reporting", - "once_cell", - "proc-macro2", - "quote", - "scratch", - "syn 2.0.15", -] - -[[package]] -name = "cxxbridge-flags" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7944172ae7e4068c533afbb984114a56c46e9ccddda550499caa222902c7f7bb" - -[[package]] -name = "cxxbridge-macro" -version = "1.0.94" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2345488264226bf682893e25de0769f3360aac9957980ec49361b083ddaa5bc5" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.15", -] - [[package]] name = "dashmap" version = "5.4.0" @@ -1423,7 +1369,7 @@ checksum = "89ca545a94061b6365f2c7355b4b32bd20df3ff95f02da9329b34ccc3bd6ee72" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -1485,9 +1431,9 @@ checksum = "d2fabcfbdc87f4758337ca535fb41a6d701b65693ce38287d856d1674551ec9b" [[package]] name = "h2" -version = "0.3.18" +version = "0.3.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "17f8a914c2987b688368b5138aa05321db91f4090cf26118185672ad588bce21" +checksum = "d357c7ae988e7d2182f7d7871d0b963962420b0678b0997ce7de72001aeab782" dependencies = [ "bytes", "fnv", @@ -1645,10 +1591,23 @@ dependencies = [ "http", "hyper", "log", - "rustls", + "rustls 0.20.8", "rustls-native-certs", "tokio", - "tokio-rustls", + "tokio-rustls 0.23.4", +] + +[[package]] +name = "hyper-rustls" +version = "0.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "0646026eb1b3eea4cd9ba47912ea5ce9cc07713d105b1a14698f4e6433d348b7" +dependencies = [ + "http", + "hyper", + "rustls 0.21.1", + "tokio", + "tokio-rustls 0.24.0", ] [[package]] @@ -1667,12 +1626,11 @@ dependencies = [ [[package]] name = "iana-time-zone-haiku" -version = "0.1.1" +version = "0.1.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0703ae284fc167426161c2e3f1da3ea71d94b21bedbcc9494e92b28e334e3dca" +checksum = "f31827a206f56af32e590ba56d5d2d085f558508192593743f16b2306495269f" dependencies = [ - "cxx", - "cxx-build", + "cc", ] [[package]] @@ -1753,9 +1711,9 @@ dependencies = [ [[package]] name = "js-sys" -version = "0.3.62" +version = "0.3.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68c16e1bfd491478ab155fd8b4896b86f9ede344949b641e61501e07c2b8b4d5" +checksum = "2f37a4a5928311ac501dee68b3c7613a1037d0edb30c8e5427bd832d55d1b790" dependencies = [ "wasm-bindgen", ] @@ -1838,9 +1796,9 @@ checksum = "2b00cc1c228a6782d0f076e7b232802e0c5689d41bb5df366f2a6b6621cfdfe1" [[package]] name = "libm" -version = "0.2.6" +version = "0.2.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "348108ab3fba42ec82ff6e9564fc4ca0247bdccdc68dd8af9764bbc79c3c8ffb" +checksum = "f7012b1bbb0719e1097c47611d3898568c546d597c2e74d66f6087edd5233ff4" [[package]] name = "libmimalloc-sys" @@ -1852,15 +1810,6 @@ dependencies = [ "libc", ] -[[package]] -name = "link-cplusplus" -version = "1.0.8" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ecd207c9c713c34f95a097a5b029ac2ce6010530c7b49d7fea24d977dede04f5" -dependencies = [ - "cc", -] - [[package]] name = "linux-raw-sys" version = "0.3.7" @@ -2268,22 +2217,22 @@ dependencies = [ [[package]] name = "pin-project" -version = "1.0.12" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ad29a609b6bcd67fee905812e544992d216af9d755757c05ed2d0e15a74c6ecc" +checksum = "c95a7476719eab1e366eaf73d0260af3021184f18177925b07f54b30089ceead" dependencies = [ "pin-project-internal", ] [[package]] name = "pin-project-internal" -version = "1.0.12" +version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "069bdb1e05adc7a8990dce9cc75370895fbe4e3d58b9b73bf1aee56359344a55" +checksum = "39407670928234ebc5e6e580247dd567ad73a3578460c5990f9503df207e8f07" dependencies = [ "proc-macro2", "quote", - "syn 1.0.109", + "syn 2.0.16", ] [[package]] @@ -2342,9 +2291,9 @@ checksum = "dc375e1527247fe1a97d8b7156678dfe7c1af2fc075c9a4db3690ecd2a148068" [[package]] name = "proc-macro2" -version = "1.0.56" +version = "1.0.58" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "2b63bdb0cd06f1f4dedf69b254734f9b45af66e4a031e42a7480257d9898b435" +checksum = "fa1fb82fc0c281dd9671101b66b771ebbe1eaf967b96ac8740dcba4b70005ca8" dependencies = [ "unicode-ident", ] @@ -2456,9 +2405,9 @@ checksum = "a5996294f19bd3aae0453a862ad728f60e6600695733dd5df01da90c54363a3c" [[package]] name = "reqwest" -version = "0.11.17" +version = "0.11.18" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "13293b639a097af28fc8a90f22add145a9c954e49d77da06263d58cf44d5fb91" +checksum = "cde824a14b7c14f85caff81225f411faacc04a2013f41670f41443742b1c1c55" dependencies = [ "base64", "bytes", @@ -2469,7 +2418,7 @@ dependencies = [ "http", "http-body", "hyper", - "hyper-rustls", + "hyper-rustls 0.24.0", "ipnet", "js-sys", "log", @@ -2477,13 +2426,13 @@ dependencies = [ "once_cell", "percent-encoding", "pin-project-lite", - "rustls", + "rustls 0.21.1", "rustls-pemfile", "serde", "serde_json", "serde_urlencoded", "tokio", - "tokio-rustls", + "tokio-rustls 0.24.0", "tokio-util", "tower-service", "url", @@ -2545,6 +2494,18 @@ dependencies = [ "webpki", ] +[[package]] +name = "rustls" +version = "0.21.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c911ba11bc8433e811ce56fde130ccf32f5127cab0e0194e9c68c5a5b671791e" +dependencies = [ + "log", + "ring", + "rustls-webpki", + "sct", +] + [[package]] name = "rustls-native-certs" version = "0.6.2" @@ -2566,6 +2527,16 @@ dependencies = [ "base64", ] +[[package]] +name = "rustls-webpki" +version = "0.100.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "d6207cd5ed3d8dca7816f8f3725513a34609c0c765bf652b8c3cb4cfd87db46b" +dependencies = [ + "ring", + "untrusted", +] + [[package]] name = "rustversion" version = "1.0.12" @@ -2625,12 +2596,6 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d29ab0c6d3fc0ee92fe66e2d99f700eab17a8d57d1c1d3b748380fb20baa78cd" -[[package]] -name = "scratch" -version = "1.0.5" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1792db035ce95be60c3f8853017b3999209281c24e2ba5bc8e59bf97a0c590c1" - [[package]] name = "sct" version = "0.7.0" @@ -2643,9 +2608,9 @@ dependencies = [ [[package]] name = "security-framework" -version = "2.8.2" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a332be01508d814fed64bf28f798a146d73792121129962fdf335bb3c49a4254" +checksum = "ca2855b3715770894e67cbfa3df957790aa0c9edc3bf06efa1a84d77fa0839d1" dependencies = [ "bitflags", "core-foundation", @@ -2656,9 +2621,9 @@ dependencies = [ [[package]] name = "security-framework-sys" -version = "2.8.0" +version = "2.9.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "31c9bb296072e961fcbd8853511dd39c2d8be2deb1e17c6860b1d30732b323b4" +checksum = "f51d0c0d83bec45f16480d0ce0058397a69e48fcdc52d1dc8855fb68acbd31a7" dependencies = [ "core-foundation-sys", "libc", @@ -2678,22 +2643,22 @@ checksum = "e6b44e8fc93a14e66336d230954dda83d18b4605ccace8fe09bc7514a71ad0bc" [[package]] name = "serde" -version = "1.0.162" +version = "1.0.163" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "71b2f6e1ab5c2b98c05f0f35b236b22e8df7ead6ffbf51d7808da7f8817e7ab6" +checksum = "2113ab51b87a539ae008b5c6c02dc020ffa39afd2d83cffcb3f4eb2722cebec2" dependencies = [ "serde_derive", ] [[package]] name = "serde_derive" -version = "1.0.162" +version = "1.0.163" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a2a0814352fd64b58489904a44ea8d90cb1a91dcb6b4f5ebabc32c8318e93cb6" +checksum = "8c805777e3930c8883389c602315a24224bcc738b63905ef87cd1420353ea93e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -2798,7 +2763,7 @@ checksum = "6e63cff320ae2c57904679ba7cb63280a3dc4613885beafb148ee7bf9aa9042d" [[package]] name = "sqlparser" version = "0.33.0" -source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=1c4c3b8#1c4c3b8e7380d640c3271a61c12de171f87c29bb" +source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=b759a95#b759a95a47e2f587d9217e2778636586234f479f" dependencies = [ "log", "sqlparser_derive", @@ -2807,7 +2772,7 @@ dependencies = [ [[package]] name = "sqlparser_derive" version = "0.1.1" -source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=1c4c3b8#1c4c3b8e7380d640c3271a61c12de171f87c29bb" +source = "git+https://github.com/synnada-ai/sqlparser-rs.git?rev=b759a95#b759a95a47e2f587d9217e2778636586234f479f" dependencies = [ "proc-macro2", "quote", @@ -2870,9 +2835,9 @@ dependencies = [ [[package]] name = "syn" -version = "2.0.15" +version = "2.0.16" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a34fcf3e8b60f57e6a14301a2e916d323af98b0ea63c599441eec8558660c822" +checksum = "a6f671d4b5ffdb8eadec19c0ae67fe2639df8684bd7bc4b83d986b8db549cf01" dependencies = [ "proc-macro2", "quote", @@ -2924,7 +2889,7 @@ checksum = "f9456a42c5b0d803c8cd86e73dd7cc9edd429499f37a3550d286d5e86720569f" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -3014,7 +2979,7 @@ checksum = "630bdcf245f78637c13ec01ffae6187cca34625e8c63150d424b59e55af2675e" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] @@ -3023,11 +2988,21 @@ version = "0.23.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c43ee83903113e03984cb9e5cebe6c04a5116269e900e3ddba8f068a62adda59" dependencies = [ - "rustls", + "rustls 0.20.8", "tokio", "webpki", ] +[[package]] +name = "tokio-rustls" +version = "0.24.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "e0d409377ff5b1e3ca6437aa86c1eb7d40c134bfec254e44c830defa92669db5" +dependencies = [ + "rustls 0.21.1", + "tokio", +] + [[package]] name = "tokio-stream" version = "0.1.14" @@ -3102,14 +3077,14 @@ checksum = "0f57e3ca2a01450b1a921183a9c9cbfda207fd822cef4ccb00a65402cbba7a74" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", ] [[package]] name = "tracing-core" -version = "0.1.30" +version = "0.1.31" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "24eb03ba0eab1fd845050058ce5e616558e8f8d8fca633e6b163fe25c797213a" +checksum = "0955b8137a1df6f1a2e9a37d8a6656291ff0297c1a97c24e0d8425fe2312f79a" dependencies = [ "once_cell", ] @@ -3200,9 +3175,9 @@ checksum = "711b9620af191e0cdc7468a8d14e709c3dcdb115b36f838e601583af800a370a" [[package]] name = "uuid" -version = "1.3.2" +version = "1.3.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4dad5567ad0cf5b760e5665964bec1b47dfd077ba8a2544b513f3556d3d239a2" +checksum = "345444e32442451b267fc254ae85a209c64be56d2890e601a0c37ff0c3c5ecd2" dependencies = [ "getrandom", ] @@ -3247,9 +3222,9 @@ checksum = "9c8d87e72b64a3b4db28d11ce29237c246188f4f51057d65a7eab63b7987e423" [[package]] name = "wasm-bindgen" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b6cb788c4e39112fbe1822277ef6fb3c55cd86b95cb3d3c4c1c9597e4ac74b4" +checksum = "5bba0e8cb82ba49ff4e229459ff22a191bbe9a1cb3a341610c9c33efc27ddf73" dependencies = [ "cfg-if", "wasm-bindgen-macro", @@ -3257,24 +3232,24 @@ dependencies = [ [[package]] name = "wasm-bindgen-backend" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "35e522ed4105a9d626d885b35d62501b30d9666283a5c8be12c14a8bdafe7822" +checksum = "19b04bc93f9d6bdee709f6bd2118f57dd6679cf1176a1af464fca3ab0d66d8fb" dependencies = [ "bumpalo", "log", "once_cell", "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-futures" -version = "0.4.35" +version = "0.4.36" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "083abe15c5d88556b77bdf7aef403625be9e327ad37c62c4e4129af740168163" +checksum = "2d1985d03709c53167ce907ff394f5316aa22cb4e12761295c5dc57dacb6297e" dependencies = [ "cfg-if", "js-sys", @@ -3284,9 +3259,9 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "358a79a0cb89d21db8120cbfb91392335913e4890665b1a7981d9e956903b434" +checksum = "14d6b024f1a526bb0234f52840389927257beb670610081360e5a03c5df9c258" dependencies = [ "quote", "wasm-bindgen-macro-support", @@ -3294,22 +3269,22 @@ dependencies = [ [[package]] name = "wasm-bindgen-macro-support" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4783ce29f09b9d93134d41297aded3a712b7b979e9c6f28c32cb88c973a94869" +checksum = "e128beba882dd1eb6200e1dc92ae6c5dbaa4311aa7bb211ca035779e5efc39f8" dependencies = [ "proc-macro2", "quote", - "syn 2.0.15", + "syn 2.0.16", "wasm-bindgen-backend", "wasm-bindgen-shared", ] [[package]] name = "wasm-bindgen-shared" -version = "0.2.85" +version = "0.2.86" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a901d592cafaa4d711bc324edfaff879ac700b19c3dfd60058d2b445be2691eb" +checksum = "ed9d5b4305409d1fc9482fee2d7f9bcbf24b3972bf59817ef757e23982242a93" [[package]] name = "wasm-streams" @@ -3326,9 +3301,9 @@ dependencies = [ [[package]] name = "web-sys" -version = "0.3.62" +version = "0.3.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "16b5f940c7edfdc6d12126d98c9ef4d1b3d470011c47c76a6581df47ad9ba721" +checksum = "3bdd9ef4e984da1187bf8110c5cf5b845fbc87a23602cdf912386a76fcd3a7c2" dependencies = [ "js-sys", "wasm-bindgen", diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 327dedc7a2ca..d9b21ffeef59 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -569,8 +569,10 @@ enum AggregateFunction { BIT_XOR = 21; BOOL_AND = 22; BOOL_OR = 23; - FIRST = 24; - LAST = 25; + // since function with same names exists in builtin window functions + // because of name scope rules, we append their name with _AGG identifiers + FIRST_VALUE_AGG = 24; + LAST_VALUE_AGG = 25; } message AggregateExprNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index af3db63d2f61..bcb521854b09 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -463,8 +463,8 @@ impl serde::Serialize for AggregateFunction { Self::BitXor => "BIT_XOR", Self::BoolAnd => "BOOL_AND", Self::BoolOr => "BOOL_OR", - Self::First => "FIRST", - Self::Last => "LAST", + Self::FirstValueAgg => "FIRST_VALUE_AGG", + Self::LastValueAgg => "LAST_VALUE_AGG", }; serializer.serialize_str(variant) } @@ -500,8 +500,8 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "BIT_XOR", "BOOL_AND", "BOOL_OR", - "FIRST", - "LAST", + "FIRST_VALUE_AGG", + "LAST_VALUE_AGG", ]; struct GeneratedVisitor; @@ -568,8 +568,8 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "BIT_XOR" => Ok(AggregateFunction::BitXor), "BOOL_AND" => Ok(AggregateFunction::BoolAnd), "BOOL_OR" => Ok(AggregateFunction::BoolOr), - "FIRST" => Ok(AggregateFunction::First), - "LAST" => Ok(AggregateFunction::Last), + "FIRST_VALUE_AGG" => Ok(AggregateFunction::FirstValueAgg), + "LAST_VALUE_AGG" => Ok(AggregateFunction::LastValueAgg), _ => Err(serde::de::Error::unknown_variant(value, FIELDS)), } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 1ff79cc11f0f..115ef9d26118 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2398,8 +2398,10 @@ pub enum AggregateFunction { BitXor = 21, BoolAnd = 22, BoolOr = 23, - First = 24, - Last = 25, + /// since function with same names exists in builtin window functions + /// because of name scope rules, we append their name with _AGG identifiers + FirstValueAgg = 24, + LastValueAgg = 25, } impl AggregateFunction { /// String value of the enum field names used in the ProtoBuf definition. @@ -2434,8 +2436,8 @@ impl AggregateFunction { AggregateFunction::BitXor => "BIT_XOR", AggregateFunction::BoolAnd => "BOOL_AND", AggregateFunction::BoolOr => "BOOL_OR", - AggregateFunction::First => "FIRST", - AggregateFunction::Last => "LAST", + AggregateFunction::FirstValueAgg => "FIRST_VALUE_AGG", + AggregateFunction::LastValueAgg => "LAST_VALUE_AGG", } } /// Creates an enum from field names used in the ProtoBuf definition. @@ -2467,8 +2469,8 @@ impl AggregateFunction { "BIT_XOR" => Some(Self::BitXor), "BOOL_AND" => Some(Self::BoolAnd), "BOOL_OR" => Some(Self::BoolOr), - "FIRST" => Some(Self::First), - "LAST" => Some(Self::Last), + "FIRST_VALUE_AGG" => Some(Self::FirstValueAgg), + "LAST_VALUE_AGG" => Some(Self::LastValueAgg), _ => None, } } diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index 762a73bd4919..1150220bef4a 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -529,8 +529,8 @@ impl From for AggregateFunction { protobuf::AggregateFunction::ApproxMedian => Self::ApproxMedian, protobuf::AggregateFunction::Grouping => Self::Grouping, protobuf::AggregateFunction::Median => Self::Median, - protobuf::AggregateFunction::First => Self::FirstValue, - protobuf::AggregateFunction::Last => Self::LastValue, + protobuf::AggregateFunction::FirstValueAgg => Self::FirstValue, + protobuf::AggregateFunction::LastValueAgg => Self::LastValue, } } } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 6f9120a4a7c0..191c49194407 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -388,8 +388,8 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::ApproxMedian => Self::ApproxMedian, AggregateFunction::Grouping => Self::Grouping, AggregateFunction::Median => Self::Median, - AggregateFunction::FirstValue => Self::First, - AggregateFunction::LastValue => Self::Last, + AggregateFunction::FirstValue => Self::FirstValueAgg, + AggregateFunction::LastValue => Self::LastValueAgg, } } } @@ -669,8 +669,12 @@ impl TryFrom<&Expr> for protobuf::LogicalExprNode { } AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, AggregateFunction::Median => protobuf::AggregateFunction::Median, - AggregateFunction::FirstValue => protobuf::AggregateFunction::First, - AggregateFunction::LastValue => protobuf::AggregateFunction::Last, + AggregateFunction::FirstValue => { + protobuf::AggregateFunction::FirstValueAgg + } + AggregateFunction::LastValue => { + protobuf::AggregateFunction::LastValueAgg + } }; let aggregate_expr = protobuf::AggregateExprNode { diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 4112d22bc17c..f296ffb344a3 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -116,7 +116,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { .map(|e| self.order_by_to_sort_expr(e, schema, planner_context)) .collect::>>()?; let order_by = (!order_by.is_empty()).then_some(order_by); - println!("order by:{:?}", order_by); let (fun, args) = self.aggregate_fn_to_expr(fun, function.args, schema, planner_context)?; return Ok(Expr::AggregateFunction(expr::AggregateFunction::new( From e5a2776e8dd5fafe466dccd137e06b1500ceaad7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 18 May 2023 17:37:12 +0300 Subject: [PATCH 41/53] Minor changes --- datafusion/expr/src/aggregate_function.rs | 4 ++-- datafusion/expr/src/window_function.rs | 3 +++ datafusion/sql/src/expr/mod.rs | 2 -- docs/source/user-guide/sql/aggregate_functions.md | 12 ++++++------ 4 files changed, 11 insertions(+), 10 deletions(-) diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index c620f353931a..d20a830c28a2 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -43,9 +43,9 @@ pub enum AggregateFunction { ApproxDistinct, /// array_agg ArrayAgg, - /// first + /// first_value FirstValue, - /// last + /// last_value LastValue, /// Variance (Sample) Variance, diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index a256f2da1c3e..29495b2f1609 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -43,6 +43,9 @@ pub enum WindowFunction { /// Find DataFusion's built-in window function by name. pub fn find_df_window_func(name: &str) -> Option { let name = name.to_lowercase(); + // We first search for window functions (They may have specialized implementations for windows with same name) + // Since the requirements for aggregators and window functions may be quite different, the same function may have different + // implementation. if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { Some(WindowFunction::BuiltInWindowFunction(built_in_function)) } else if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { diff --git a/datafusion/sql/src/expr/mod.rs b/datafusion/sql/src/expr/mod.rs index 9394000be24c..b914149cf4e1 100644 --- a/datafusion/sql/src/expr/mod.rs +++ b/datafusion/sql/src/expr/mod.rs @@ -299,8 +299,6 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { SQLExpr::Subquery(subquery) => self.parse_scalar_subquery(*subquery, schema, planner_context), SQLExpr::ArrayAgg(array_agg) => self.parse_array_agg(array_agg, schema, planner_context), - // SQLExpr::FIRST(first_agg) => self.parse_first_last_agg(FirstLastAgg::First( first_agg), schema, planner_context), - // SQLExpr::LAST(last_agg) => self.parse_first_last_agg(FirstLastAgg::Last(last_agg), schema, planner_context), _ => Err(DataFusionError::NotImplemented(format!( "Unsupported ast node in sqltorel: {sql:?}" diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index 8ae3e5e6f6e2..d7c65c256dd5 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -36,8 +36,8 @@ Aggregate functions operate on a set of values to compute a single result. - [min](#min) - [sum](#sum) - [array_agg](#array_agg) -- [first](#first) -- [last](#last) +- [first_value](#first_value) +- [last_value](#last_value) ### `avg` @@ -207,12 +207,12 @@ array_agg(expression [ORDER BY expression]) - **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -### `first` +### `first_value` Returns the first entry in aggregation group. If ordering requirement is given, returns the first element of the ordered group. ``` -first(expression [ORDER BY expression]) +first_value(expression [ORDER BY expression]) ``` #### Arguments @@ -220,12 +220,12 @@ first(expression [ORDER BY expression]) - **expression**: Expression to operate on. Can be a constant, column, or function, and any combination of arithmetic operators. -### `last` +### `last_value` Returns the last entry in aggregation group. If ordering requirement is given, returns the last element of the ordered group. ``` -last(expression [ORDER BY expression]) +last_value(expression [ORDER BY expression]) ``` #### Arguments From af4d20900cf258fea66418e01de37e6b770f260a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 22 May 2023 10:49:51 +0300 Subject: [PATCH 42/53] do not consider ordering requirement in ordering insensitive aggregators --- .../core/src/physical_plan/aggregates/mod.rs | 22 ++++++++++++++++++- .../sqllogictests/test_files/groupby.slt | 13 ++++++----- datafusion/expr/src/aggregate_function.rs | 4 ++-- datafusion/expr/src/utils.rs | 2 +- datafusion/expr/src/window_function.rs | 4 ++-- 5 files changed, 33 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 247dfc27784e..63d41db8cec6 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -56,6 +56,7 @@ mod utils; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; +use datafusion_physical_expr::expressions::{ArrayAgg, FirstAgg, LastAgg}; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -389,6 +390,13 @@ fn get_finest_requirement< Ok(result) } +// Check whether aggregate function is ordering sensitive +fn is_ordering_sensitive(aggr_expr: &Arc) -> bool { + aggr_expr.as_any().is::() + || aggr_expr.as_any().is::() + || aggr_expr.as_any().is::() +} + impl AggregateExec { /// Create a new hash aggregate execution plan pub fn try_new( @@ -396,7 +404,7 @@ impl AggregateExec { group_by: PhysicalGroupBy, aggr_expr: Vec>, filter_expr: Vec>>, - order_by_expr: Vec>>, + mut order_by_expr: Vec>>, input: Arc, input_schema: SchemaRef, ) -> Result { @@ -414,6 +422,18 @@ impl AggregateExec { // 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 { + order_by_expr = aggr_expr + .iter() + .zip(order_by_expr.into_iter()) + .map(|(aggr_expr, fn_reqs)| { + // If aggregation function is ordering sensitive, keep ordering requirement as is otherwise ignore requirement + if is_ordering_sensitive(aggr_expr) { + fn_reqs + } else { + None + } + }) + .collect::>(); let requirement = get_finest_requirement( &order_by_expr, || input.equivalence_properties(), diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 7d898d8c54c8..03efb05143c2 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -1974,25 +1974,26 @@ query III # test_source_sorted_groupby2 - +# If ordering is not important for the aggregation function, we should ignore the ordering requirement. Hence +# "ORDER BY a DESC" should have no effect. query TT EXPLAIN SELECT a, d, - SUM(c) as summation1 + SUM(c ORDER BY a DESC) as summation1 FROM annotated_data_infinite2 GROUP BY d, a ---- logical_plan -Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.c) AS summation1 ---Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(annotated_data_infinite2.c)]] +Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST] AS summation1 +--Aggregate: groupBy=[[annotated_data_infinite2.d, annotated_data_infinite2.a]], aggr=[[SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]]] ----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan -ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c)@2 as summation1] +ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] --AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallyOrdered ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true query III SELECT a, d, - SUM(c) as summation1 + SUM(c ORDER BY a DESC) as summation1 FROM annotated_data_infinite2 GROUP BY d, a ---- diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index d20a830c28a2..1471d6ac2d87 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -17,7 +17,7 @@ //! Aggregate function module contains all built-in aggregate functions definitions -use crate::utils::convert_camel_uppercase_snake; +use crate::utils::convert_camel_to_upper_snake; use crate::{type_coercion::aggregates::*, Signature, TypeSignature, Volatility}; use arrow::datatypes::{DataType, Field}; use datafusion_common::{DataFusionError, Result}; @@ -86,7 +86,7 @@ impl fmt::Display for AggregateFunction { // uppercase of the debug. // Convert Camel form to uppercase snake // such as FirstValue => FIRST_VALUE - write!(f, "{}", convert_camel_uppercase_snake(format!("{self:?}"))) + write!(f, "{}", convert_camel_to_upper_snake(format!("{self:?}"))) } } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 6f5a707d5fcd..65f773c9d1e7 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -56,7 +56,7 @@ pub fn exprlist_to_columns(expr: &[Expr], accum: &mut HashSet) -> Result /// Convert Camel form to uppercase snake /// such as FirstValue => FIRST_VALUE -pub fn convert_camel_uppercase_snake(in_str: String) -> String { +pub fn convert_camel_to_upper_snake(in_str: String) -> String { let mut res = String::new(); for ch in in_str.chars() { if ch.is_uppercase() && !res.is_empty() { diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 29495b2f1609..d48d3aacdda4 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -23,7 +23,7 @@ use crate::aggregate_function::AggregateFunction; use crate::type_coercion::functions::data_types; -use crate::utils::convert_camel_uppercase_snake; +use crate::utils::convert_camel_to_upper_snake; use crate::{aggregate_function, AggregateUDF, Signature, TypeSignature, Volatility}; use arrow::datatypes::DataType; use datafusion_common::{DataFusionError, Result}; @@ -57,7 +57,7 @@ pub fn find_df_window_func(name: &str) -> Option { impl fmt::Display for BuiltInWindowFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", convert_camel_uppercase_snake(format!("{self:?}"))) + write!(f, "{}", convert_camel_to_upper_snake(format!("{self:?}"))) } } From eacfaff6b0195e93cba81ef714b8e108fb039bd7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 22 May 2023 11:06:58 +0300 Subject: [PATCH 43/53] Reject aggregate order by for window functions. --- datafusion/core/src/physical_plan/aggregates/mod.rs | 2 +- datafusion/core/tests/sqllogictests/test_files/window.slt | 6 ++++++ datafusion/sql/src/expr/function.rs | 6 ++++++ 3 files changed, 13 insertions(+), 1 deletion(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 63d41db8cec6..b7e4cea03039 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -426,7 +426,7 @@ impl AggregateExec { .iter() .zip(order_by_expr.into_iter()) .map(|(aggr_expr, fn_reqs)| { - // If aggregation function is ordering sensitive, keep ordering requirement as is otherwise ignore requirement + // If aggregation function is ordering sensitive, keep ordering requirement as is; otherwise ignore requirement if is_ordering_sensitive(aggr_expr) { fn_reqs } else { diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index 2d78798ecc1c..bdb5c6ae0a52 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -3017,6 +3017,12 @@ SELECT a, b, c, 0 0 3 11 96 11 2 10 36 10 36 11 5 11 9 0 0 4 9 72 9 NULL 14 45 14 45 9 4 9 9 +#fn aggregate order by with window frame +# in window expressions, aggregate function should not have itself ordering requirement, (requirements should be defined in window clause) +# hence query below, should return error (Similar to what PostgreSQL does). +statement error DataFusion error: Error during planning: Aggregate ORDER BY is not implemented for window functions +SELECT SUM(inc_col ORDER BY a DESC) OVER() as last + FROM annotated_data_infinite2 statement ok drop table annotated_data_finite2 diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index f296ffb344a3..f05ffe7f338a 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -53,6 +53,12 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { return Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))); }; + if !function.order_by.is_empty() && function.over.is_some() { + return Err(DataFusionError::Plan( + "Aggregate ORDER BY is not implemented for window functions".to_string(), + )); + } + // then, window function if let Some(window) = function.over.take() { let partition_by = window From 4ce1038ff4fb832db40f681f63f261eb28a5ec75 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 23 May 2023 13:00:10 +0300 Subject: [PATCH 44/53] simplifications --- datafusion/common/Cargo.toml | 4 ---- datafusion/core/Cargo.toml | 4 ---- datafusion/expr/Cargo.toml | 4 ---- datafusion/expr/src/utils.rs | 5 ++--- datafusion/expr/src/window_function.rs | 2 +- .../physical-expr/src/aggregate/first_last.rs | 22 ------------------- datafusion/sql/Cargo.toml | 4 ---- 7 files changed, 3 insertions(+), 42 deletions(-) diff --git a/datafusion/common/Cargo.toml b/datafusion/common/Cargo.toml index 2d6a954e1567..ad7cabf580b6 100644 --- a/datafusion/common/Cargo.toml +++ b/datafusion/common/Cargo.toml @@ -47,10 +47,6 @@ num_cpus = "1.13.0" object_store = { version = "0.5.4", default-features = false, optional = true } parquet = { workspace = true, optional = true } pyo3 = { version = "0.18.0", optional = true } -# sqlparser = "0.33" -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } [dev-dependencies] rand = "0.8.4" diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index d150339cd9bd..a72d314bce4c 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -85,10 +85,6 @@ percent-encoding = "2.2.0" pin-project-lite = "^0.2.7" rand = "0.8" smallvec = { version = "1.6", features = ["union"] } -# sqlparser = { version = "0.33", features = ["visitor"] } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support", features = ["visitor"] } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de", features = ["visitor"] } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } tempfile = "3" tokio = { version = "1.0", features = ["macros", "rt", "rt-multi-thread", "sync", "fs", "parking_lot"] } diff --git a/datafusion/expr/Cargo.toml b/datafusion/expr/Cargo.toml index e754b04d5b41..eda9bb85f45a 100644 --- a/datafusion/expr/Cargo.toml +++ b/datafusion/expr/Cargo.toml @@ -38,10 +38,6 @@ path = "src/lib.rs" ahash = { version = "0.8", default-features = false, features = ["runtime-rng"] } arrow = { workspace = true } datafusion-common = { path = "../common", version = "25.0.0" } -# sqlparser = "0.33" -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } [dev-dependencies] ctor = "0.2.0" diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 65f773c9d1e7..8f9967528fc1 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -60,10 +60,9 @@ pub fn convert_camel_to_upper_snake(in_str: String) -> String { let mut res = String::new(); for ch in in_str.chars() { if ch.is_uppercase() && !res.is_empty() { - res = format!("{}_{}", res, ch.to_uppercase()); - } else { - res = format!("{}{}", res, ch.to_uppercase()); + res.push('_'); } + res.push_str(format!("{}", ch.to_uppercase()).as_str()); } res } diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index d48d3aacdda4..ead80b092a0e 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -45,7 +45,7 @@ pub fn find_df_window_func(name: &str) -> Option { let name = name.to_lowercase(); // We first search for window functions (They may have specialized implementations for windows with same name) // Since the requirements for aggregators and window functions may be quite different, the same function may have different - // implementation. + // implementation. If function is not found among specialized window functions search them among AggregateFunctions. if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { Some(WindowFunction::BuiltInWindowFunction(built_in_function)) } else if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 34c6eb568a1c..f27c4a20bea0 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -43,7 +43,6 @@ impl FirstAgg { name: impl Into, data_type: DataType, ) -> Self { - println!("first agg init"); Self { name: name.into(), data_type, @@ -82,10 +81,6 @@ impl AggregateExpr for FirstAgg { &self.name } - fn supports_bounded_execution(&self) -> bool { - true - } - fn reverse_expr(&self) -> Option> { Some(Arc::new(LastAgg::new( self.expr.clone(), @@ -143,12 +138,6 @@ impl Accumulator for FirstAccumulator { Ok(()) } - fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let values = &values[0]; - self.count -= (values.len() - values.null_count()) as u64; - Ok(()) - } - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { // FIRST(first1, first2, first3, ...) self.update_batch(states) @@ -183,7 +172,6 @@ impl LastAgg { name: impl Into, data_type: DataType, ) -> Self { - println!("last agg init"); Self { name: name.into(), data_type, @@ -222,10 +210,6 @@ impl AggregateExpr for LastAgg { &self.name } - fn supports_bounded_execution(&self) -> bool { - true - } - fn reverse_expr(&self) -> Option> { Some(Arc::new(FirstAgg::new( self.expr.clone(), @@ -285,12 +269,6 @@ impl Accumulator for LastAccumulator { Ok(()) } - fn retract_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - let values = &values[0]; - self.count -= (values.len() - values.null_count()) as u64; - Ok(()) - } - fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { // LAST(last1, last2, last3, ...) self.update_batch(states) diff --git a/datafusion/sql/Cargo.toml b/datafusion/sql/Cargo.toml index bccb9a1ad310..1a9ae7da4111 100644 --- a/datafusion/sql/Cargo.toml +++ b/datafusion/sql/Cargo.toml @@ -42,10 +42,6 @@ arrow-schema = { workspace = true } datafusion-common = { path = "../common", version = "25.0.0" } datafusion-expr = { path = "../expr", version = "25.0.0" } log = "^0.4" -# sqlparser = "0.33" -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", branch="feature/first_last_support"} -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "870b3de" } -# sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "1c4c3b8", features = ["visitor"] } sqlparser = { git = "https://github.com/synnada-ai/sqlparser-rs.git", rev = "b759a95", features = ["visitor"] } [dev-dependencies] ctor = "0.2.0" From 1bdda3856f095fd825784fd186cc8fc112288dd9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 23 May 2023 16:14:06 +0300 Subject: [PATCH 45/53] Fix cargo lock file --- datafusion-cli/Cargo.lock | 34 ++++++++++++++-------------------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index ab33ba0b9873..03968e0f5566 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -610,9 +610,9 @@ dependencies = [ [[package]] name = "base64" -version = "0.21.0" +version = "0.21.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a4a4ddaa51a5bc52a6948f74c06d20aaaddb71924eab79b8c97a8c556e942d6a" +checksum = "3f1e31e207a6b8fb791a38ea3105e6cb541f55e4d029902d3039a4ad07cc4105" [[package]] name = "base64-simd" @@ -685,9 +685,9 @@ dependencies = [ [[package]] name = "bumpalo" -version = "3.12.2" +version = "3.13.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3c6ed94e98ecff0c12dd1b04c15ec0d7d9458ca8fe806cea6f12954efe74c63b" +checksum = "a3e2c3daef883ecc1b5d58c15adae93470a91d425f3532ba1695849656af3fc1" [[package]] name = "byteorder" @@ -1131,9 +1131,9 @@ dependencies = [ [[package]] name = "digest" -version = "0.10.6" +version = "0.10.7" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8168378f4e5023e7218c89c891c0fd8ecdb5e5e4f18cb78f38cf245dd021e76f" +checksum = "9ed9a281f7bc9b7576e61468ba615a66a5c8cfdff42420a70aa82701a3b1e292" dependencies = [ "block-buffer", "crypto-common", @@ -1812,9 +1812,9 @@ dependencies = [ [[package]] name = "linux-raw-sys" -version = "0.3.7" +version = "0.3.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ece97ea872ece730aed82664c424eb4c8291e1ff2480247ccf7409044bc6479f" +checksum = "ef53942eb7bf7ff43a617b3e2c1c4a5ecf5944a7c1bc12d7ee39bbb15e5c1519" [[package]] name = "lock_api" @@ -2388,9 +2388,9 @@ dependencies = [ [[package]] name = "regex" -version = "1.8.1" +version = "1.8.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "af83e617f331cc6ae2da5443c602dfa5af81e517212d9d611a5b3ba1777b5370" +checksum = "d1a59b5d8e97dee33696bf13c5ba8ab85341c002922fba050069326b9c498974" dependencies = [ "aho-corasick", "memchr", @@ -2399,9 +2399,9 @@ dependencies = [ [[package]] name = "regex-syntax" -version = "0.7.1" +version = "0.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a5996294f19bd3aae0453a862ad728f60e6600695733dd5df01da90c54363a3c" +checksum = "436b050e76ed2903236f032a59761c1eb99e1b0aead2c257922771dab1fc8c78" [[package]] name = "reqwest" @@ -2608,15 +2608,9 @@ dependencies = [ [[package]] name = "security-framework" -<<<<<<< HEAD -version = "2.9.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ca2855b3715770894e67cbfa3df957790aa0c9edc3bf06efa1a84d77fa0839d1" -======= version = "2.9.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1fc758eb7bffce5b308734e9b0c1468893cae9ff70ebf13e7090be8dcbcc83a8" ->>>>>>> main dependencies = [ "bitflags", "core-foundation", @@ -2824,9 +2818,9 @@ dependencies = [ [[package]] name = "subtle" -version = "2.4.1" +version = "2.5.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "6bdef32e8150c2a081110b42772ffe7d7c9032b606bc226c8260fd97e0976601" +checksum = "81cdd64d312baedb58e21336b31bc043b77e01cc99033ce76ef539f78e965ebc" [[package]] name = "syn" From 94fe486a100b7a36ffddac0d4b7146d034944716 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 24 May 2023 12:59:16 +0300 Subject: [PATCH 46/53] Update comment --- datafusion/core/src/physical_plan/aggregates/mod.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index b63bef8fbbb3..f157105d6984 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -390,6 +390,9 @@ fn get_finest_requirement< } // Check whether aggregate function is ordering sensitive +// Ordering sensitive means that, the order data is fed to the aggregator, affects the result of aggregator. +// For instance, `SUM` aggregator doesn't depend on the order of the inputs as long as same set of values is used during computation, +// However, `FirstAgg` depends on the order (if order changes, first value in the list would change). fn is_ordering_sensitive(aggr_expr: &Arc) -> bool { aggr_expr.as_any().is::() || aggr_expr.as_any().is::() From 13ee806656fe2015cb69855edd0a1649140e37c8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 24 May 2023 13:09:23 +0300 Subject: [PATCH 47/53] Rename aggregator first and last --- .../core/src/physical_plan/aggregates/mod.rs | 6 +-- .../physical-expr/src/aggregate/build_in.rs | 4 +- .../physical-expr/src/aggregate/first_last.rs | 53 +++++++++---------- .../physical-expr/src/expressions/mod.rs | 2 +- datafusion/sql/src/expr/function.rs | 3 ++ 5 files changed, 35 insertions(+), 33 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index f157105d6984..8da2fcde28b3 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -55,7 +55,7 @@ mod utils; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; -use datafusion_physical_expr::expressions::{ArrayAgg, FirstAgg, LastAgg}; +use datafusion_physical_expr::expressions::{ArrayAgg, FirstValue, LastValue}; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -394,8 +394,8 @@ fn get_finest_requirement< // For instance, `SUM` aggregator doesn't depend on the order of the inputs as long as same set of values is used during computation, // However, `FirstAgg` depends on the order (if order changes, first value in the list would change). fn is_ordering_sensitive(aggr_expr: &Arc) -> bool { - aggr_expr.as_any().is::() - || aggr_expr.as_any().is::() + aggr_expr.as_any().is::() + || aggr_expr.as_any().is::() || aggr_expr.as_any().is::() } diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 4853b033e577..69ff89a3929d 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -308,12 +308,12 @@ pub fn create_aggregate_expr( "MEDIAN(DISTINCT) aggregations are not available".to_string(), )); } - (AggregateFunction::FirstValue, _) => Arc::new(expressions::FirstAgg::new( + (AggregateFunction::FirstValue, _) => Arc::new(expressions::FirstValue::new( input_phy_exprs[0].clone(), name, input_phy_types[0].clone(), )), - (AggregateFunction::LastValue, _) => Arc::new(expressions::LastAgg::new( + (AggregateFunction::LastValue, _) => Arc::new(expressions::LastValue::new( input_phy_exprs[0].clone(), name, input_phy_types[0].clone(), diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index f27c4a20bea0..02e7617d0081 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -28,15 +28,15 @@ use datafusion_expr::Accumulator; use std::any::Any; use std::sync::Arc; -/// FIRST aggregate expression +/// FIRST_VALUE aggregate expression #[derive(Debug)] -pub struct FirstAgg { +pub struct FirstValue { name: String, pub data_type: DataType, expr: Arc, } -impl FirstAgg { +impl FirstValue { /// Create a new ArrayAgg aggregate function pub fn new( expr: Arc, @@ -51,7 +51,7 @@ impl FirstAgg { } } -impl AggregateExpr for FirstAgg { +impl AggregateExpr for FirstValue { /// Return a reference to Any that can be used for downcasting fn as_any(&self) -> &dyn Any { self @@ -62,12 +62,12 @@ impl AggregateExpr for FirstAgg { } fn create_accumulator(&self) -> Result> { - Ok(Box::new(FirstAccumulator::try_new(&self.data_type)?)) + Ok(Box::new(FirstValueAccumulator::try_new(&self.data_type)?)) } fn state_fields(&self) -> Result> { Ok(vec![Field::new( - format_state_name(&self.name, "first"), + format_state_name(&self.name, "first_value"), self.data_type.clone(), true, )]) @@ -82,7 +82,7 @@ impl AggregateExpr for FirstAgg { } fn reverse_expr(&self) -> Option> { - Some(Arc::new(LastAgg::new( + Some(Arc::new(LastValue::new( self.expr.clone(), self.name.clone(), self.data_type.clone(), @@ -90,11 +90,11 @@ impl AggregateExpr for FirstAgg { } fn create_sliding_accumulator(&self) -> Result> { - Ok(Box::new(FirstAccumulator::try_new(&self.data_type)?)) + Ok(Box::new(FirstValueAccumulator::try_new(&self.data_type)?)) } } -impl PartialEq for FirstAgg { +impl PartialEq for FirstValue { fn eq(&self, other: &dyn Any) -> bool { down_cast_any_ref(other) .downcast_ref::() @@ -108,12 +108,12 @@ impl PartialEq for FirstAgg { } #[derive(Debug)] -struct FirstAccumulator { +struct FirstValueAccumulator { first: ScalarValue, count: u64, } -impl FirstAccumulator { +impl FirstValueAccumulator { /// new First accumulator pub fn try_new(data_type: &DataType) -> Result { Ok(Self { @@ -123,7 +123,7 @@ impl FirstAccumulator { } } -impl Accumulator for FirstAccumulator { +impl Accumulator for FirstValueAccumulator { fn state(&self) -> Result> { Ok(vec![self.first.clone(), ScalarValue::from(self.count)]) } @@ -139,7 +139,7 @@ impl Accumulator for FirstAccumulator { } fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - // FIRST(first1, first2, first3, ...) + // FIRST_VALUE(first1, first2, first3, ...) self.update_batch(states) } @@ -157,15 +157,15 @@ impl Accumulator for FirstAccumulator { } } -/// LAST aggregate expression +/// LAST_VALUE aggregate expression #[derive(Debug)] -pub struct LastAgg { +pub struct LastValue { name: String, pub data_type: DataType, expr: Arc, } -impl LastAgg { +impl LastValue { /// Create a new ArrayAgg aggregate function pub fn new( expr: Arc, @@ -180,7 +180,7 @@ impl LastAgg { } } -impl AggregateExpr for LastAgg { +impl AggregateExpr for LastValue { /// Return a reference to Any that can be used for downcasting fn as_any(&self) -> &dyn Any { self @@ -191,12 +191,12 @@ impl AggregateExpr for LastAgg { } fn create_accumulator(&self) -> Result> { - Ok(Box::new(LastAccumulator::try_new(&self.data_type)?)) + Ok(Box::new(LastValueAccumulator::try_new(&self.data_type)?)) } fn state_fields(&self) -> Result> { Ok(vec![Field::new( - format_state_name(&self.name, "last"), + format_state_name(&self.name, "last_value"), self.data_type.clone(), true, )]) @@ -211,7 +211,7 @@ impl AggregateExpr for LastAgg { } fn reverse_expr(&self) -> Option> { - Some(Arc::new(FirstAgg::new( + Some(Arc::new(FirstValue::new( self.expr.clone(), self.name.clone(), self.data_type.clone(), @@ -219,11 +219,11 @@ impl AggregateExpr for LastAgg { } fn create_sliding_accumulator(&self) -> Result> { - Ok(Box::new(LastAccumulator::try_new(&self.data_type)?)) + Ok(Box::new(LastValueAccumulator::try_new(&self.data_type)?)) } } -impl PartialEq for LastAgg { +impl PartialEq for LastValue { fn eq(&self, other: &dyn Any) -> bool { down_cast_any_ref(other) .downcast_ref::() @@ -237,15 +237,14 @@ impl PartialEq for LastAgg { } #[derive(Debug)] -struct LastAccumulator { +struct LastValueAccumulator { last: ScalarValue, count: u64, } -impl LastAccumulator { +impl LastValueAccumulator { /// new Last accumulator pub fn try_new(data_type: &DataType) -> Result { - println!("last accumulator init"); Ok(Self { last: ScalarValue::try_from(data_type)?, count: 0, @@ -253,7 +252,7 @@ impl LastAccumulator { } } -impl Accumulator for LastAccumulator { +impl Accumulator for LastValueAccumulator { fn state(&self) -> Result> { Ok(vec![self.last.clone(), ScalarValue::from(self.count)]) } @@ -270,7 +269,7 @@ impl Accumulator for LastAccumulator { } fn merge_batch(&mut self, states: &[ArrayRef]) -> Result<()> { - // LAST(last1, last2, last3, ...) + // LAST_VALUE(last1, last2, last3, ...) self.update_batch(states) } diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 01af210a4d21..66d593c5cafa 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -54,7 +54,7 @@ pub use crate::aggregate::correlation::Correlation; pub use crate::aggregate::count::Count; pub use crate::aggregate::count_distinct::DistinctCount; pub use crate::aggregate::covariance::{Covariance, CovariancePop}; -pub use crate::aggregate::first_last::{FirstAgg, LastAgg}; +pub use crate::aggregate::first_last::{FirstValue, LastValue}; pub use crate::aggregate::grouping::Grouping; pub use crate::aggregate::median::Median; pub use crate::aggregate::min_max::{Max, Min}; diff --git a/datafusion/sql/src/expr/function.rs b/datafusion/sql/src/expr/function.rs index 70050e5d5686..682501b13444 100644 --- a/datafusion/sql/src/expr/function.rs +++ b/datafusion/sql/src/expr/function.rs @@ -53,6 +53,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { return Ok(Expr::ScalarFunction(ScalarFunction::new(fun, args))); }; + // If function is a window function (it has an OVER clause), + // it shouldn't have ordering requirement as function argument + // required ordering should be defined in OVER clause. if !function.order_by.is_empty() && function.over.is_some() { return Err(DataFusionError::Plan( "Aggregate ORDER BY is not implemented for window functions".to_string(), From 1cd494f1cb02abee279bb73198f4c69038175b6e Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 24 May 2023 13:36:59 +0300 Subject: [PATCH 48/53] minor change --- datafusion/expr/src/window_function.rs | 2 +- datafusion/physical-expr/src/aggregate/first_last.rs | 5 ++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index ead80b092a0e..b23505663abb 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -45,7 +45,7 @@ pub fn find_df_window_func(name: &str) -> Option { let name = name.to_lowercase(); // We first search for window functions (They may have specialized implementations for windows with same name) // Since the requirements for aggregators and window functions may be quite different, the same function may have different - // implementation. If function is not found among specialized window functions search them among AggregateFunctions. + // implementation. If function is not found among specialized window functions, search them among AggregateFunctions. if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { Some(WindowFunction::BuiltInWindowFunction(built_in_function)) } else if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 02e7617d0081..79aa3a394cc6 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -131,7 +131,7 @@ impl Accumulator for FirstValueAccumulator { fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it let values = &values[0]; - if self.count == 0 && values.len() > 0 { + if self.count == 0 && !values.is_empty() { self.first = ScalarValue::try_from_array(values, 0)?; } self.count += (values.len() - values.null_count()) as u64; @@ -258,9 +258,8 @@ impl Accumulator for LastValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - // If we have seen first value, we shouldn't update it let values = &values[0]; - if values.len() > 0 { + if !values.is_empty() { // Update with last value in the array. self.last = ScalarValue::try_from_array(values, values.len() - 1)?; } From f2a0a682cc71783a34614f4e3ea3405ad3c95384 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 25 May 2023 00:06:44 -0500 Subject: [PATCH 49/53] Comment improvements --- .../core/src/physical_plan/aggregates/mod.rs | 12 ++++++------ .../tests/sqllogictests/test_files/window.slt | 5 +++-- datafusion/expr/src/utils.rs | 14 +++++++------- datafusion/expr/src/window_function.rs | 8 +++++--- .../physical-expr/src/aggregate/first_last.rs | 16 +++++++++------- datafusion/proto/proto/datafusion.proto | 4 ++-- .../source/user-guide/sql/aggregate_functions.md | 4 ++-- 7 files changed, 34 insertions(+), 29 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 8da2fcde28b3..8f9071011346 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -389,11 +389,11 @@ fn get_finest_requirement< Ok(result) } -// Check whether aggregate function is ordering sensitive -// Ordering sensitive means that, the order data is fed to the aggregator, affects the result of aggregator. -// For instance, `SUM` aggregator doesn't depend on the order of the inputs as long as same set of values is used during computation, -// However, `FirstAgg` depends on the order (if order changes, first value in the list would change). -fn is_ordering_sensitive(aggr_expr: &Arc) -> bool { +/// Checks whether the given aggregate expression is order-sensitive. +/// For instance, a `SUM` aggregation doesn't depend on the order of its inputs. +/// However, a `FirstAgg` depends on the input ordering (if the order changes, +/// the first value in the list would change). +fn is_order_sensitive(aggr_expr: &Arc) -> bool { aggr_expr.as_any().is::() || aggr_expr.as_any().is::() || aggr_expr.as_any().is::() @@ -429,7 +429,7 @@ impl AggregateExec { .zip(order_by_expr.into_iter()) .map(|(aggr_expr, fn_reqs)| { // If aggregation function is ordering sensitive, keep ordering requirement as is; otherwise ignore requirement - if is_ordering_sensitive(aggr_expr) { + if is_order_sensitive(aggr_expr) { fn_reqs } else { None diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index bdb5c6ae0a52..ed5b663c0a86 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -3018,8 +3018,9 @@ SELECT a, b, c, 0 0 4 9 72 9 NULL 14 45 14 45 9 4 9 9 #fn aggregate order by with window frame -# in window expressions, aggregate function should not have itself ordering requirement, (requirements should be defined in window clause) -# hence query below, should return error (Similar to what PostgreSQL does). +# In window expressions, aggregate functions should not have an ordering requirement, such requirements +# should be defined in the window frame. Therefore, the query below should generate an error. Note that +# PostgreSQL also behaves this way. statement error DataFusion error: Error during planning: Aggregate ORDER BY is not implemented for window functions SELECT SUM(inc_col ORDER BY a DESC) OVER() as last FROM annotated_data_infinite2 diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 8f9967528fc1..107af8c0da5f 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -54,17 +54,17 @@ pub fn exprlist_to_columns(expr: &[Expr], accum: &mut HashSet) -> Result Ok(()) } -/// Convert Camel form to uppercase snake -/// such as FirstValue => FIRST_VALUE +/// Converts a camel case string to upper snake case (a.k.a. macro case). +/// Example: "FirstValue" maps to "FIRST_VALUE". pub fn convert_camel_to_upper_snake(in_str: String) -> String { - let mut res = String::new(); + let mut result = String::new(); for ch in in_str.chars() { - if ch.is_uppercase() && !res.is_empty() { - res.push('_'); + if ch.is_uppercase() && !result.is_empty() { + result.push('_'); } - res.push_str(format!("{}", ch.to_uppercase()).as_str()); + result.push_str(format!("{}", ch.to_uppercase()).as_str()); } - res + result } /// Count the number of distinct exprs in a list of group by expressions. If the diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index b23505663abb..52ddb5f47f47 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -43,9 +43,11 @@ pub enum WindowFunction { /// Find DataFusion's built-in window function by name. pub fn find_df_window_func(name: &str) -> Option { let name = name.to_lowercase(); - // We first search for window functions (They may have specialized implementations for windows with same name) - // Since the requirements for aggregators and window functions may be quite different, the same function may have different - // implementation. If function is not found among specialized window functions, search them among AggregateFunctions. + // Code paths for window functions leveraging ordinary aggregators and + // built-in window functions are quite different, and the same function + // may have different implementations for these cases. If the sought + // function is not found among built-in window functions, we search for + // it among aggregate functions. if let Ok(built_in_function) = BuiltInWindowFunction::from_str(name.as_str()) { Some(WindowFunction::BuiltInWindowFunction(built_in_function)) } else if let Ok(aggregate) = AggregateFunction::from_str(name.as_str()) { diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 79aa3a394cc6..2d049e20b79a 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -15,16 +15,18 @@ // specific language governing permissions and limitations // under the License. -//! Defines physical expressions that can evaluated at runtime during query execution +//! Defines the FIRST_VALUE/LAST_VALUE aggregations. use crate::aggregate::utils::down_cast_any_ref; use crate::expressions::format_state_name; use crate::{AggregateExpr, PhysicalExpr}; + use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; use arrow_array::Array; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Accumulator; + use std::any::Any; use std::sync::Arc; @@ -37,7 +39,7 @@ pub struct FirstValue { } impl FirstValue { - /// Create a new ArrayAgg aggregate function + /// Creates a new FIRST_VALUE aggregation function. pub fn new( expr: Arc, name: impl Into, @@ -114,10 +116,10 @@ struct FirstValueAccumulator { } impl FirstValueAccumulator { - /// new First accumulator + /// Creates a new `FirstValueAccumulator` for the given `data_type`. pub fn try_new(data_type: &DataType) -> Result { - Ok(Self { - first: ScalarValue::try_from(data_type)?, + ScalarValue::try_from(data_type).map(|value| Self { + first: value, count: 0, }) } @@ -166,7 +168,7 @@ pub struct LastValue { } impl LastValue { - /// Create a new ArrayAgg aggregate function + /// Creates a new LAST_VALUE aggregation function. pub fn new( expr: Arc, name: impl Into, @@ -243,7 +245,7 @@ struct LastValueAccumulator { } impl LastValueAccumulator { - /// new Last accumulator + /// Creates a new `LastValueAccumulator` for the given `data_type`. pub fn try_new(data_type: &DataType) -> Result { Ok(Self { last: ScalarValue::try_from(data_type)?, diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 23050e2647c6..7c35452085e3 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -570,8 +570,8 @@ enum AggregateFunction { BIT_XOR = 21; BOOL_AND = 22; BOOL_OR = 23; - // since function with same names exists in builtin window functions - // because of name scope rules, we append their name with _AGG identifiers + // When a function with the same name exists among built-in window functions, + // we append "_AGG" to obey name scoping rules. FIRST_VALUE_AGG = 24; LAST_VALUE_AGG = 25; } diff --git a/docs/source/user-guide/sql/aggregate_functions.md b/docs/source/user-guide/sql/aggregate_functions.md index d7c65c256dd5..132ba47e2461 100644 --- a/docs/source/user-guide/sql/aggregate_functions.md +++ b/docs/source/user-guide/sql/aggregate_functions.md @@ -209,7 +209,7 @@ array_agg(expression [ORDER BY expression]) ### `first_value` -Returns the first entry in aggregation group. If ordering requirement is given, returns the first element of the ordered group. +Returns the first element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group. ``` first_value(expression [ORDER BY expression]) @@ -222,7 +222,7 @@ first_value(expression [ORDER BY expression]) ### `last_value` -Returns the last entry in aggregation group. If ordering requirement is given, returns the last element of the ordered group. +Returns the last element in an aggregation group according to the requested ordering. If no ordering is given, returns an arbitrary element from the group. ``` last_value(expression [ORDER BY expression]) From c4a47e2f29bb0f2a817f8c9f01a9f9459880acbd Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 May 2023 09:49:38 +0300 Subject: [PATCH 50/53] Remove count from First,Last accumulators --- .../physical-expr/src/aggregate/first_last.rs | 28 ++++--------------- datafusion/proto/src/generated/prost.rs | 4 +-- 2 files changed, 8 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 2d049e20b79a..f65360c75199 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -112,31 +112,26 @@ impl PartialEq for FirstValue { #[derive(Debug)] struct FirstValueAccumulator { first: ScalarValue, - count: u64, } impl FirstValueAccumulator { /// Creates a new `FirstValueAccumulator` for the given `data_type`. pub fn try_new(data_type: &DataType) -> Result { - ScalarValue::try_from(data_type).map(|value| Self { - first: value, - count: 0, - }) + ScalarValue::try_from(data_type).map(|value| Self { first: value }) } } impl Accumulator for FirstValueAccumulator { fn state(&self) -> Result> { - Ok(vec![self.first.clone(), ScalarValue::from(self.count)]) + Ok(vec![self.first.clone()]) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it let values = &values[0]; - if self.count == 0 && !values.is_empty() { + if !values.is_empty() { self.first = ScalarValue::try_from_array(values, 0)?; } - self.count += (values.len() - values.null_count()) as u64; Ok(()) } @@ -146,11 +141,7 @@ impl Accumulator for FirstValueAccumulator { } fn evaluate(&self) -> Result { - if self.count == 0 { - ScalarValue::try_from(&self.first.get_datatype()) - } else { - Ok(self.first.clone()) - } + Ok(self.first.clone()) } fn size(&self) -> usize { @@ -241,7 +232,6 @@ impl PartialEq for LastValue { #[derive(Debug)] struct LastValueAccumulator { last: ScalarValue, - count: u64, } impl LastValueAccumulator { @@ -249,14 +239,13 @@ impl LastValueAccumulator { pub fn try_new(data_type: &DataType) -> Result { Ok(Self { last: ScalarValue::try_from(data_type)?, - count: 0, }) } } impl Accumulator for LastValueAccumulator { fn state(&self) -> Result> { - Ok(vec![self.last.clone(), ScalarValue::from(self.count)]) + Ok(vec![self.last.clone()]) } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { @@ -265,7 +254,6 @@ impl Accumulator for LastValueAccumulator { // Update with last value in the array. self.last = ScalarValue::try_from_array(values, values.len() - 1)?; } - self.count += (values.len() - values.null_count()) as u64; Ok(()) } @@ -275,11 +263,7 @@ impl Accumulator for LastValueAccumulator { } fn evaluate(&self) -> Result { - if self.count == 0 { - ScalarValue::try_from(&self.last.get_datatype()) - } else { - Ok(self.last.clone()) - } + Ok(self.last.clone()) } fn size(&self) -> usize { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 9745125bf141..7e48db10f3ae 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2400,8 +2400,8 @@ pub enum AggregateFunction { BitXor = 21, BoolAnd = 22, BoolOr = 23, - /// since function with same names exists in builtin window functions - /// because of name scope rules, we append their name with _AGG identifiers + /// When a function with the same name exists among built-in window functions, + /// we append "_AGG" to obey name scoping rules. FirstValueAgg = 24, LastValueAgg = 25, } From 18935b28996d8e3d51f3610df3fb7db11960f5bb Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 26 May 2023 21:31:36 +0300 Subject: [PATCH 51/53] Address reviews --- .../sqllogictests/test_files/groupby.slt | 27 +++++++++++++++++++ .../tests/sqllogictests/test_files/window.slt | 10 +++++-- 2 files changed, 35 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/groupby.slt b/datafusion/core/tests/sqllogictests/test_files/groupby.slt index 06e7d615f25e..8bac60bbba8a 100644 --- a/datafusion/core/tests/sqllogictests/test_files/groupby.slt +++ b/datafusion/core/tests/sqllogictests/test_files/groupby.slt @@ -2060,6 +2060,33 @@ SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c 1 2 74 1 3 99 +# when LAST_VALUE, or FIRST_VALUE value do not contain ordering requirement +# queries should still work, However, result depends on the scanning order and +# not deterministic +query TT +EXPLAIN SELECT a, b, LAST_VALUE(c) as last_c + FROM annotated_data_infinite2 + GROUP BY a, b +---- +logical_plan +Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(annotated_data_infinite2.c) AS last_c +--Aggregate: groupBy=[[annotated_data_infinite2.a, annotated_data_infinite2.b]], aggr=[[LAST_VALUE(annotated_data_infinite2.c)]] +----TableScan: annotated_data_infinite2 projection=[a, b, c] +physical_plan +ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true + +query III +SELECT a, b, LAST_VALUE(c) as last_c + FROM annotated_data_infinite2 + GROUP BY a, b +---- +0 0 24 +0 1 49 +1 2 74 +1 3 99 + statement ok drop table annotated_data_infinite2; diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index 9ff7a980a3d7..1dbcb97a0f8a 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -3022,7 +3022,13 @@ SELECT a, b, c, # should be defined in the window frame. Therefore, the query below should generate an error. Note that # PostgreSQL also behaves this way. statement error DataFusion error: Error during planning: Aggregate ORDER BY is not implemented for window functions -SELECT SUM(inc_col ORDER BY a DESC) OVER() as last +SELECT SUM(b ORDER BY a ASC) OVER() as sum1 + FROM annotated_data_infinite2 + +# Even if, requirement of window clause and aggregate function match; +# we should raise an error, when an ordering requirement is given to aggregate functions in window clauses. +statement error DataFusion error: Error during planning: Aggregate ORDER BY is not implemented for window functions +SELECT a, b, LAST_VALUE(c ORDER BY a ASC) OVER (order by a ASC) as last_c FROM annotated_data_infinite2 statement ok @@ -3142,4 +3148,4 @@ SELECT WINDOW window1 AS (ORDER BY C12), window1 AS (ORDER BY C3) ORDER BY C3 - LIMIT 5 \ No newline at end of file + LIMIT 5 From 48057ccb9bbb5d821b0dd8ce80748f8b863dfd49 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 26 May 2023 22:02:25 +0300 Subject: [PATCH 52/53] Remove camel to upper snake util, make aggregate function names explicit --- datafusion/expr/src/aggregate_function.rs | 40 ++++++++++++++++++++--- datafusion/expr/src/utils.rs | 13 -------- datafusion/expr/src/window_function.rs | 22 +++++++++++-- 3 files changed, 55 insertions(+), 20 deletions(-) diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 1471d6ac2d87..8258c8b80585 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -17,7 +17,6 @@ //! Aggregate function module contains all built-in aggregate functions definitions -use crate::utils::convert_camel_to_upper_snake; use crate::{type_coercion::aggregates::*, Signature, TypeSignature, Volatility}; use arrow::datatypes::{DataType, Field}; use datafusion_common::{DataFusionError, Result}; @@ -81,12 +80,43 @@ pub enum AggregateFunction { BoolOr, } +impl AggregateFunction { + fn name(&self) -> &str { + use AggregateFunction::*; + match self { + Count => "COUNT", + Sum => "SUM", + Min => "MIN", + Max => "MAX", + Avg => "AVG", + Median => "MEDIAN", + ApproxDistinct => "APPROX_DISTINCT", + ArrayAgg => "ARRAY_AGG", + FirstValue => "FIRST_VALUE", + LastValue => "LAST_VALUE", + Variance => "VARIANCE", + VariancePop => "VARIANCE_POP", + Stddev => "STDDEV", + StddevPop => "STDDEV_POP", + Covariance => "COVARIANCE", + CovariancePop => "COVARIANCE_POP", + Correlation => "CORRELATION", + ApproxPercentileCont => "APPROX_PERCENTILE_CONT", + ApproxPercentileContWithWeight => "APPROX_PERCENTILE_CONT_WITH_WEIGHT", + ApproxMedian => "APPROX_MEDIAN", + Grouping => "GROUPING", + BitAnd => "BIT_AND", + BitOr => "BIT_OR", + BitXor => "BIT_XOR", + BoolAnd => "BOOL_AND", + BoolOr => "BOOL_OR", + } + } +} + impl fmt::Display for AggregateFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - // uppercase of the debug. - // Convert Camel form to uppercase snake - // such as FirstValue => FIRST_VALUE - write!(f, "{}", convert_camel_to_upper_snake(format!("{self:?}"))) + write!(f, "{}", self.name()) } } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 107af8c0da5f..1d7aa536f910 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -54,19 +54,6 @@ pub fn exprlist_to_columns(expr: &[Expr], accum: &mut HashSet) -> Result Ok(()) } -/// Converts a camel case string to upper snake case (a.k.a. macro case). -/// Example: "FirstValue" maps to "FIRST_VALUE". -pub fn convert_camel_to_upper_snake(in_str: String) -> String { - let mut result = String::new(); - for ch in in_str.chars() { - if ch.is_uppercase() && !result.is_empty() { - result.push('_'); - } - result.push_str(format!("{}", ch.to_uppercase()).as_str()); - } - result -} - /// Count the number of distinct exprs in a list of group by expressions. If the /// first element is a `GroupingSet` expression then it must be the only expr. pub fn grouping_set_expr_count(group_expr: &[Expr]) -> Result { diff --git a/datafusion/expr/src/window_function.rs b/datafusion/expr/src/window_function.rs index 52ddb5f47f47..1bae3a162e50 100644 --- a/datafusion/expr/src/window_function.rs +++ b/datafusion/expr/src/window_function.rs @@ -23,7 +23,6 @@ use crate::aggregate_function::AggregateFunction; use crate::type_coercion::functions::data_types; -use crate::utils::convert_camel_to_upper_snake; use crate::{aggregate_function, AggregateUDF, Signature, TypeSignature, Volatility}; use arrow::datatypes::DataType; use datafusion_common::{DataFusionError, Result}; @@ -59,7 +58,7 @@ pub fn find_df_window_func(name: &str) -> Option { impl fmt::Display for BuiltInWindowFunction { fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result { - write!(f, "{}", convert_camel_to_upper_snake(format!("{self:?}"))) + write!(f, "{}", self.name()) } } @@ -106,6 +105,25 @@ pub enum BuiltInWindowFunction { NthValue, } +impl BuiltInWindowFunction { + fn name(&self) -> &str { + use BuiltInWindowFunction::*; + match self { + RowNumber => "ROW_NUMBER", + Rank => "RANK", + DenseRank => "DENSE_RANK", + PercentRank => "PERCENT_RANK", + CumeDist => "CUME_DIST", + Ntile => "NTILE", + Lag => "LAG", + Lead => "LEAD", + FirstValue => "FIRST_VALUE", + LastValue => "LAST_VALUE", + NthValue => "NTH_VALUE", + } + } +} + impl FromStr for BuiltInWindowFunction { type Err = DataFusionError; fn from_str(name: &str) -> Result { From 5cc149aaa03b68f906f6d30000437593de859ac0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 26 May 2023 22:13:29 +0300 Subject: [PATCH 53/53] update the test --- datafusion/core/tests/sqllogictests/test_files/window.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index 1dbcb97a0f8a..8ab9b29da4dc 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -3028,7 +3028,7 @@ SELECT SUM(b ORDER BY a ASC) OVER() as sum1 # Even if, requirement of window clause and aggregate function match; # we should raise an error, when an ordering requirement is given to aggregate functions in window clauses. statement error DataFusion error: Error during planning: Aggregate ORDER BY is not implemented for window functions -SELECT a, b, LAST_VALUE(c ORDER BY a ASC) OVER (order by a ASC) as last_c +EXPLAIN SELECT a, b, LAST_VALUE(c ORDER BY a ASC) OVER (order by a ASC) as last_c FROM annotated_data_infinite2 statement ok