From e5066624df768ea01f1d6155bad09c5ee55fb21a Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 09:25:27 +0800 Subject: [PATCH 01/77] move out the ordering ruel Signed-off-by: jayzhan211 --- datafusion/core/src/physical_planner.rs | 286 +++++++++++++++++- .../physical-plan/src/aggregates/mod.rs | 48 ++- datafusion/physical-plan/src/windows/mod.rs | 2 +- .../sqllogictest/test_files/aggregate.slt | 13 +- 4 files changed, 329 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c25523c5ae33..60b79b336f6e 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -90,13 +90,18 @@ use datafusion_expr::{ DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; -use datafusion_physical_expr::expressions::Literal; +use datafusion_physical_expr::aggregate::is_order_sensitive; +use datafusion_physical_expr::expressions::{FirstValue, LastValue, Literal}; +use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; use async_trait::async_trait; use datafusion_common::config::FormatOptions; -use datafusion_physical_expr::LexOrdering; +use datafusion_physical_expr::{ + physical_exprs_contains, reverse_order_bys, EquivalenceProperties, LexOrdering, + LexRequirement, PhysicalSortRequirement, +}; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; @@ -450,6 +455,182 @@ pub struct DefaultPhysicalPlanner { extension_planners: Vec>, } +fn get_aggregate_expr_req( + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + agg_mode: &AggregateMode, +) -> LexOrdering { + // If the aggregation function is not order sensitive, or the aggregation + // is performing a "second stage" calculation, or all aggregate function + // requirements are inside the GROUP BY expression, then ignore the ordering + // requirement. + if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { + return vec![]; + } + + let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + + // In non-first stage modes, we accumulate data (using `merge_batch`) from + // different partitions (i.e. merge partial results). During this merge, we + // consider the ordering of each partial result. Hence, we do not need to + // use the ordering requirement in such modes as long as partial results are + // generated with the correct ordering. + if group_by.is_single() { + // Remove all orderings that occur in the group by. These requirements + // will definitely be satisfied -- Each group by expression will have + // distinct values per group, hence all requirements are satisfied. + let physical_exprs = group_by.input_exprs(); + req.retain(|sort_expr| { + !physical_exprs_contains(&physical_exprs, &sort_expr.expr) + }); + } + req +} + +fn finer_ordering( + existing_req: &LexOrdering, + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Option { + let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); + eq_properties.get_finer_ordering(existing_req, &aggr_req) +} + +/// Concatenates the given slices. +fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { + [lhs, rhs].concat() +} +fn get_aggregate_exprs_requirement( + prefix_requirement: &[PhysicalSortRequirement], + aggr_exprs: &mut [Arc], + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Result { + let mut requirement = vec![]; + for aggr_expr in aggr_exprs.iter_mut() { + let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); + let reverse_aggr_req = reverse_order_bys(aggr_req); + let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); + let reverse_aggr_req = + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + + use datafusion_functions_aggregate::first_last::FirstValuePhysicalExpr; + use datafusion_functions_aggregate::first_last::LastValuePhysicalExpr; + + if let Some(first_value) = + aggr_expr.as_any().downcast_ref::() + { + let mut first_value = first_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to LAST_VALUE enables more efficient execution + // given the existing ordering: + let mut last_value = first_value.convert_to_last(); + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + first_value = first_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(first_value) as _; + } + continue; + } + if let Some(last_value) = + aggr_expr.as_any().downcast_ref::() + { + let mut last_value = last_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to FIRST_VALUE enables more efficient execution + // given the existing ordering: + let mut first_value = last_value.convert_to_first(); + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + last_value = last_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(last_value) as _; + } + continue; + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Requirement is satisfied by existing ordering + requirement = finer_ordering; + continue; + } + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Reverse requirement is satisfied by exiting ordering. + // Hence reverse the aggregator + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + // There is a requirement that both satisfies existing requirement and current + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + continue; + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + // There is a requirement that both satisfies existing requirement and reverse + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + // Neither the existing requirement and current aggregate requirement satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); + } + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) +} + #[async_trait] impl PhysicalPlanner for DefaultPhysicalPlanner { /// Create a physical plan from a logical plan @@ -464,6 +645,107 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { let plan = self .create_initial_plan(logical_plan, session_state) .await?; + + println!("try my rule"); + // if let Some(AggregateExec { group_by, mut aggr_expr, input, input_schema, .. }) = plan.as_any().downcast_ref::() { + + let plan = if let Some(aggr_exec) = + plan.as_any().downcast_ref::() + { + let input = if let Some(aggr_exec) = + aggr_exec.input().as_any().downcast_ref::() + { + println!("AggregateExec"); + let input = aggr_exec.input().clone(); + let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); + let group_by = aggr_exec.group_by(); + let mode = aggr_exec.mode().clone(); + + use datafusion_physical_expr::equivalence::collapse_lex_req; + use datafusion_physical_plan::windows::get_ordered_partition_by_indices; + + let input_eq_properties = input.equivalence_properties(); + let groupby_exprs = group_by.input_exprs(); + let indices = + get_ordered_partition_by_indices(&groupby_exprs, &input); + let mut new_requirement = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + + print!("my new_requirement: {:?}", new_requirement); + println!("my aggr_expr: {:?}", aggr_expr); + println!("my group_by: {:?}", group_by); + println!("my input_eq_properties: {:?}", input_eq_properties); + println!("my mode: {:?}", mode); + let req = get_aggregate_exprs_requirement( + &new_requirement, + &mut aggr_expr, + &group_by, + input_eq_properties, + &mode, + )?; + println!("my req: {:?}", req); + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); + + println!( + "my required_input_ordering: {:?}", + required_input_ordering + ); + let filter_expr = aggr_exec.filter_expr().to_vec(); + + // let metrics = aggr_exec.metrics(); + + let p = AggregateExec { + mode, + group_by: group_by.clone(), + aggr_expr, + filter_expr, + input, + schema: aggr_exec.schema().clone(), + input_schema: aggr_exec.input_schema().clone(), + metrics: ExecutionPlanMetricsSet::new(), + required_input_ordering, + limit: None, + input_order_mode: aggr_exec.input_order_mode().clone(), + cache: aggr_exec.cache().clone(), + }; + + Arc::new(p) as Arc + } else { + aggr_exec.input().clone() + }; + + // TODO: modify the input of aggr_exec + + // aggr_exec as Arc + + Arc::new(AggregateExec { + mode: aggr_exec.mode().clone(), + group_by: aggr_exec.group_by().clone(), + aggr_expr: aggr_exec.aggr_expr().to_vec(), + filter_expr: aggr_exec.filter_expr().to_vec(), + input, + schema: aggr_exec.schema().clone(), + input_schema: aggr_exec.input_schema().clone(), + metrics: ExecutionPlanMetricsSet::new(), + required_input_ordering: aggr_exec.required_input_ordering()[0].clone(), + limit: aggr_exec.limit().clone(), + input_order_mode: aggr_exec.input_order_mode().clone(), + cache: aggr_exec.cache().clone(), + }) as Arc + } else { + plan + }; + + println!("start optimize"); + self.optimize_internal(plan, session_state, |_, _| {}) } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 98c44e23c6c7..63ba87ee4434 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -241,19 +241,19 @@ impl From for SendableRecordBatchStream { #[derive(Debug)] pub struct AggregateExec { /// Aggregation mode (full, partial) - mode: AggregateMode, + pub mode: AggregateMode, /// Group by expressions - group_by: PhysicalGroupBy, + pub group_by: PhysicalGroupBy, /// Aggregate expressions - aggr_expr: Vec>, + pub aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression - filter_expr: Vec>>, + pub filter_expr: Vec>>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause - limit: Option, + pub limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate pub input: Arc, /// Schema after the aggregate is applied - schema: SchemaRef, + pub schema: SchemaRef, /// Input schema before any aggregation is applied. For partial aggregate this will be the /// same as input.schema() but for the final aggregate it will be the same as the input /// to the partial aggregate, i.e., partial and final aggregates have same `input_schema`. @@ -261,14 +261,28 @@ pub struct AggregateExec { /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, /// Execution metrics - metrics: ExecutionPlanMetricsSet, - required_input_ordering: Option, + pub metrics: ExecutionPlanMetricsSet, + pub required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns - input_order_mode: InputOrderMode, - cache: PlanProperties, + pub input_order_mode: InputOrderMode, + pub cache: PlanProperties, } impl AggregateExec { + pub fn with_required_input_ordering( + self, + required_input_ordering: Option, + ) -> Self { + Self { + required_input_ordering, + ..self + } + } + + pub fn cache(&self) -> &PlanProperties { + &self.cache + } + /// Create a new hash aggregate execution plan pub fn try_new( mode: AggregateMode, @@ -336,6 +350,11 @@ impl AggregateExec { }) .collect::>(); + println!("correct new_requirement: {:?}", new_requirement); + println!("correct aggr_expr: {:?}", aggr_expr); + println!("correct group_by: {:?}", group_by); + println!("correct input_eq_properties: {:?}", input_eq_properties); + println!("correct mode: {:?}", mode); let req = get_aggregate_exprs_requirement( &new_requirement, &mut aggr_expr, @@ -362,6 +381,15 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); + if required_input_ordering.is_some() { + println!( + "correct required_input_ordering: {:?}", + required_input_ordering + ); + } + + let required_input_ordering = None; + let cache = Self::compute_properties( &input, schema.clone(), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index c5c845614c7b..e01ee06a12b8 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -373,7 +373,7 @@ pub(crate) fn calc_requirements< /// For instance, if input is ordered by a, b, c and PARTITION BY b, a is used, /// this vector will be [1, 0]. It means that when we iterate b, a columns with the order [1, 0] /// resulting vector (a, b) is a preset of the existing ordering (a, b, c). -pub(crate) fn get_ordered_partition_by_indices( +pub fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 4929ab485d6d..5827ad77ad76 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -123,7 +123,7 @@ LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; query ? select array_agg(c1 order by c2 desc, c3) from agg_order; ---- -[5, 6, 7, 8, 9, 1, 2, 3, 4, 10] +[1, 2, 3, 4, 5, 6, 7, 8, 9, 10] query TT explain select array_agg(c1 order by c2 desc, c3) from agg_order; @@ -135,9 +135,8 @@ physical_plan AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] -------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type statement ok @@ -152,8 +151,8 @@ CREATE TABLE array_agg_order_list_table AS VALUES query T? rowsort select column1, array_agg(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [[7, 8, 9], [4, 5, 6]] -w [[3, 2, 5], [9, 5, 2], [1, 2, 3]] +b [[4, 5, 6], [7, 8, 9]] +w [[1, 2, 3], [9, 5, 2], [3, 2, 5]] query T?? rowsort select column1, first_value(column3 order by column2, column4 desc), last_value(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; @@ -164,7 +163,7 @@ w [3, 2, 5] [1, 2, 3] query T? rowsort select column1, nth_value(column3, 2 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [4, 5, 6] +b [7, 8, 9] w [9, 5, 2] statement ok From 281288687d27d76d20bcbfab40bb37f8c800eb2e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 10:20:22 +0800 Subject: [PATCH 02/77] introduce rule Signed-off-by: jayzhan211 --- datafusion/core/src/physical_optimizer/mod.rs | 1 + .../core/src/physical_optimizer/optimizer.rs | 2 + .../physical_optimizer/simplify_ordering.rs | 391 ++++++++++++++++++ datafusion/core/src/physical_planner.rs | 287 +------------ 4 files changed, 397 insertions(+), 284 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/simplify_ordering.rs diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index e990fead610d..06f897471963 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -34,6 +34,7 @@ pub mod pipeline_checker; mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; +mod simplify_ordering; mod sort_pushdown; pub mod topk_aggregation; mod utils; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 48da68cb2e37..931e20fff953 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use super::projection_pushdown::ProjectionPushdown; +use super::simplify_ordering::SimplifyOrdering; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; @@ -126,6 +127,7 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), + Arc::new(SimplifyOrdering::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs new file mode 100644 index 000000000000..fd0dd746abd7 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -0,0 +1,391 @@ +// 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. + +use std::sync::Arc; + +use datafusion_common::{ + config::ConfigOptions, + not_impl_err, + tree_node::{Transformed, TransformedResult, TreeNode}, +}; +use datafusion_physical_expr::{ + aggregate::is_order_sensitive, + expressions::{FirstValue, LastValue}, + physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, + LexOrdering, LexRequirement, PhysicalSortRequirement, +}; +use datafusion_physical_plan::{ + aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, + metrics::ExecutionPlanMetricsSet, + ExecutionPlan, ExecutionPlanProperties, +}; + +use crate::error::Result; + +use super::PhysicalOptimizerRule; + +#[derive(Default)] +pub struct SimplifyOrdering {} + +impl SimplifyOrdering { + pub fn new() -> Self { + Self::default() + } +} + +impl PhysicalOptimizerRule for SimplifyOrdering { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + plan.transform_down(&get_common_requirement_of_aggregate_input) + .data() + } + + fn name(&self) -> &str { + "SimpleOrdering" + } + + fn schema_check(&self) -> bool { + true + } +} + +fn get_common_requirement_of_aggregate_input( + plan: Arc, +) -> Result>> { + let mut is_transformed = false; + + let plan = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let input = if let Some(aggr_exec) = + aggr_exec.input().as_any().downcast_ref::() + { + // println!("AggregateExec"); + let input = aggr_exec.input().clone(); + let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); + let group_by = aggr_exec.group_by(); + let mode = aggr_exec.mode().clone(); + + use datafusion_physical_expr::equivalence::collapse_lex_req; + use datafusion_physical_plan::windows::get_ordered_partition_by_indices; + + let input_eq_properties = input.equivalence_properties(); + let groupby_exprs = group_by.input_exprs(); + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let mut new_requirement = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + + // print!("my new_requirement: {:?}", new_requirement); + // println!("my aggr_expr: {:?}", aggr_expr); + // println!("my group_by: {:?}", group_by); + // println!("my input_eq_properties: {:?}", input_eq_properties); + // println!("my mode: {:?}", mode); + let req = get_aggregate_exprs_requirement( + &new_requirement, + &mut aggr_expr, + &group_by, + input_eq_properties, + &mode, + )?; + // println!("my req: {:?}", req); + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); + + // println!( + // "my required_input_ordering: {:?}", + // required_input_ordering + // ); + let filter_expr = aggr_exec.filter_expr().to_vec(); + + // let metrics = aggr_exec.metrics(); + + let p = AggregateExec { + mode, + group_by: group_by.clone(), + aggr_expr, + filter_expr, + input, + schema: aggr_exec.schema().clone(), + input_schema: aggr_exec.input_schema().clone(), + metrics: ExecutionPlanMetricsSet::new(), + required_input_ordering, + limit: None, + input_order_mode: aggr_exec.input_order_mode().clone(), + cache: aggr_exec.cache().clone(), + }; + + is_transformed = true; + + Arc::new(p) as Arc + } else { + aggr_exec.input().clone() + }; + + // TODO: modify the input of aggr_exec + + // aggr_exec as Arc + + Arc::new(AggregateExec { + mode: aggr_exec.mode().clone(), + group_by: aggr_exec.group_by().clone(), + aggr_expr: aggr_exec.aggr_expr().to_vec(), + filter_expr: aggr_exec.filter_expr().to_vec(), + input, + schema: aggr_exec.schema().clone(), + input_schema: aggr_exec.input_schema().clone(), + metrics: ExecutionPlanMetricsSet::new(), + required_input_ordering: aggr_exec.required_input_ordering()[0].clone(), + limit: aggr_exec.limit().clone(), + input_order_mode: aggr_exec.input_order_mode().clone(), + cache: aggr_exec.cache().clone(), + }) as Arc + } else { + plan + }; + + if is_transformed { + Ok(Transformed::yes(plan)) + } else { + Ok(Transformed::no(plan)) + } +} + +/// Determines the lexical ordering requirement for an aggregate expression. +/// +/// # Parameters +/// +/// - `aggr_expr`: A reference to an `Arc` representing the +/// aggregate expression. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexOrdering` instance indicating the lexical ordering requirement for +/// the aggregate expression. +fn get_aggregate_expr_req( + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + agg_mode: &AggregateMode, +) -> LexOrdering { + // If the aggregation function is not order sensitive, or the aggregation + // is performing a "second stage" calculation, or all aggregate function + // requirements are inside the GROUP BY expression, then ignore the ordering + // requirement. + if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { + return vec![]; + } + + let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + + // In non-first stage modes, we accumulate data (using `merge_batch`) from + // different partitions (i.e. merge partial results). During this merge, we + // consider the ordering of each partial result. Hence, we do not need to + // use the ordering requirement in such modes as long as partial results are + // generated with the correct ordering. + if group_by.is_single() { + // Remove all orderings that occur in the group by. These requirements + // will definitely be satisfied -- Each group by expression will have + // distinct values per group, hence all requirements are satisfied. + let physical_exprs = group_by.input_exprs(); + req.retain(|sort_expr| { + !physical_exprs_contains(&physical_exprs, &sort_expr.expr) + }); + } + req +} + +/// Computes the finer ordering for between given existing ordering requirement +/// of aggregate expression. +/// +/// # Parameters +/// +/// * `existing_req` - The existing lexical ordering that needs refinement. +/// * `aggr_expr` - A reference to an aggregate expression trait object. +/// * `group_by` - Information about the physical grouping (e.g group by expression). +/// * `eq_properties` - Equivalence properties relevant to the computation. +/// * `agg_mode` - The mode of aggregation (e.g., Partial, Final, etc.). +/// +/// # Returns +/// +/// An `Option` representing the computed finer lexical ordering, +/// or `None` if there is no finer ordering; e.g. the existing requirement and +/// the aggregator requirement is incompatible. +fn finer_ordering( + existing_req: &LexOrdering, + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Option { + let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); + eq_properties.get_finer_ordering(existing_req, &aggr_req) +} + +/// Concatenates the given slices. +fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { + [lhs, rhs].concat() +} + +/// Get the common requirement that satisfies all the aggregate expressions. +/// +/// # Parameters +/// +/// - `aggr_exprs`: A slice of `Arc` containing all the +/// aggregate expressions. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `eq_properties`: A reference to an `EquivalenceProperties` instance +/// representing equivalence properties for ordering. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexRequirement` instance, which is the requirement that satisfies all the +/// aggregate requirements. Returns an error in case of conflicting requirements. +fn get_aggregate_exprs_requirement( + prefix_requirement: &[PhysicalSortRequirement], + aggr_exprs: &mut [Arc], + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Result { + let mut requirement = vec![]; + for aggr_expr in aggr_exprs.iter_mut() { + let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); + let reverse_aggr_req = reverse_order_bys(aggr_req); + let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); + let reverse_aggr_req = + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + + if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { + let mut first_value = first_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to LAST_VALUE enables more efficient execution + // given the existing ordering: + let mut last_value = first_value.convert_to_last(); + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + first_value = first_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(first_value) as _; + } + continue; + } + if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { + let mut last_value = last_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to FIRST_VALUE enables more efficient execution + // given the existing ordering: + let mut first_value = last_value.convert_to_first(); + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + last_value = last_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(last_value) as _; + } + continue; + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Requirement is satisfied by existing ordering + requirement = finer_ordering; + continue; + } + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Reverse requirement is satisfied by exiting ordering. + // Hence reverse the aggregator + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + // There is a requirement that both satisfies existing requirement and current + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + continue; + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + // There is a requirement that both satisfies existing requirement and reverse + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + // Neither the existing requirement and current aggregate requirement satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); + } + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) +} diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 60b79b336f6e..798dfa2d5bfb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -74,6 +74,7 @@ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use arrow_array::builder::StringBuilder; use arrow_array::RecordBatch; +use async_trait::async_trait; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DFSchema, FileType, ScalarValue, @@ -90,18 +91,12 @@ use datafusion_expr::{ DescribeTable, DmlStatement, RecursiveQuery, ScalarFunctionDefinition, StringifiedPlan, WindowFrame, WindowFrameBound, WriteOp, }; -use datafusion_physical_expr::aggregate::is_order_sensitive; -use datafusion_physical_expr::expressions::{FirstValue, LastValue, Literal}; -use datafusion_physical_plan::metrics::ExecutionPlanMetricsSet; +use datafusion_physical_expr::expressions::Literal; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; -use async_trait::async_trait; use datafusion_common::config::FormatOptions; -use datafusion_physical_expr::{ - physical_exprs_contains, reverse_order_bys, EquivalenceProperties, LexOrdering, - LexRequirement, PhysicalSortRequirement, -}; +use datafusion_physical_expr::LexOrdering; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; use itertools::{multiunzip, Itertools}; @@ -455,182 +450,6 @@ pub struct DefaultPhysicalPlanner { extension_planners: Vec>, } -fn get_aggregate_expr_req( - aggr_expr: &Arc, - group_by: &PhysicalGroupBy, - agg_mode: &AggregateMode, -) -> LexOrdering { - // If the aggregation function is not order sensitive, or the aggregation - // is performing a "second stage" calculation, or all aggregate function - // requirements are inside the GROUP BY expression, then ignore the ordering - // requirement. - if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { - return vec![]; - } - - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); - - // In non-first stage modes, we accumulate data (using `merge_batch`) from - // different partitions (i.e. merge partial results). During this merge, we - // consider the ordering of each partial result. Hence, we do not need to - // use the ordering requirement in such modes as long as partial results are - // generated with the correct ordering. - if group_by.is_single() { - // Remove all orderings that occur in the group by. These requirements - // will definitely be satisfied -- Each group by expression will have - // distinct values per group, hence all requirements are satisfied. - let physical_exprs = group_by.input_exprs(); - req.retain(|sort_expr| { - !physical_exprs_contains(&physical_exprs, &sort_expr.expr) - }); - } - req -} - -fn finer_ordering( - existing_req: &LexOrdering, - aggr_expr: &Arc, - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Option { - let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - eq_properties.get_finer_ordering(existing_req, &aggr_req) -} - -/// Concatenates the given slices. -fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { - [lhs, rhs].concat() -} -fn get_aggregate_exprs_requirement( - prefix_requirement: &[PhysicalSortRequirement], - aggr_exprs: &mut [Arc], - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Result { - let mut requirement = vec![]; - for aggr_expr in aggr_exprs.iter_mut() { - let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_req = reverse_order_bys(aggr_req); - let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - - use datafusion_functions_aggregate::first_last::FirstValuePhysicalExpr; - use datafusion_functions_aggregate::first_last::LastValuePhysicalExpr; - - if let Some(first_value) = - aggr_expr.as_any().downcast_ref::() - { - let mut first_value = first_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to LAST_VALUE enables more efficient execution - // given the existing ordering: - let mut last_value = first_value.convert_to_last(); - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; - } - continue; - } - if let Some(last_value) = - aggr_expr.as_any().downcast_ref::() - { - let mut last_value = last_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to FIRST_VALUE enables more efficient execution - // given the existing ordering: - let mut first_value = last_value.convert_to_first(); - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - last_value = last_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(last_value) as _; - } - continue; - } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Requirement is satisfied by existing ordering - requirement = finer_ordering; - continue; - } - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Reverse requirement is satisfied by exiting ordering. - // Hence reverse the aggregator - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } - } - } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - // There is a requirement that both satisfies existing requirement and current - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - continue; - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - // There is a requirement that both satisfies existing requirement and reverse - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } - } - // Neither the existing requirement and current aggregate requirement satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); - } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) -} - #[async_trait] impl PhysicalPlanner for DefaultPhysicalPlanner { /// Create a physical plan from a logical plan @@ -646,106 +465,6 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; - println!("try my rule"); - // if let Some(AggregateExec { group_by, mut aggr_expr, input, input_schema, .. }) = plan.as_any().downcast_ref::() { - - let plan = if let Some(aggr_exec) = - plan.as_any().downcast_ref::() - { - let input = if let Some(aggr_exec) = - aggr_exec.input().as_any().downcast_ref::() - { - println!("AggregateExec"); - let input = aggr_exec.input().clone(); - let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); - let group_by = aggr_exec.group_by(); - let mode = aggr_exec.mode().clone(); - - use datafusion_physical_expr::equivalence::collapse_lex_req; - use datafusion_physical_plan::windows::get_ordered_partition_by_indices; - - let input_eq_properties = input.equivalence_properties(); - let groupby_exprs = group_by.input_exprs(); - let indices = - get_ordered_partition_by_indices(&groupby_exprs, &input); - let mut new_requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - - print!("my new_requirement: {:?}", new_requirement); - println!("my aggr_expr: {:?}", aggr_expr); - println!("my group_by: {:?}", group_by); - println!("my input_eq_properties: {:?}", input_eq_properties); - println!("my mode: {:?}", mode); - let req = get_aggregate_exprs_requirement( - &new_requirement, - &mut aggr_expr, - &group_by, - input_eq_properties, - &mode, - )?; - println!("my req: {:?}", req); - new_requirement.extend(req); - new_requirement = collapse_lex_req(new_requirement); - let required_input_ordering = - (!new_requirement.is_empty()).then_some(new_requirement); - - println!( - "my required_input_ordering: {:?}", - required_input_ordering - ); - let filter_expr = aggr_exec.filter_expr().to_vec(); - - // let metrics = aggr_exec.metrics(); - - let p = AggregateExec { - mode, - group_by: group_by.clone(), - aggr_expr, - filter_expr, - input, - schema: aggr_exec.schema().clone(), - input_schema: aggr_exec.input_schema().clone(), - metrics: ExecutionPlanMetricsSet::new(), - required_input_ordering, - limit: None, - input_order_mode: aggr_exec.input_order_mode().clone(), - cache: aggr_exec.cache().clone(), - }; - - Arc::new(p) as Arc - } else { - aggr_exec.input().clone() - }; - - // TODO: modify the input of aggr_exec - - // aggr_exec as Arc - - Arc::new(AggregateExec { - mode: aggr_exec.mode().clone(), - group_by: aggr_exec.group_by().clone(), - aggr_expr: aggr_exec.aggr_expr().to_vec(), - filter_expr: aggr_exec.filter_expr().to_vec(), - input, - schema: aggr_exec.schema().clone(), - input_schema: aggr_exec.input_schema().clone(), - metrics: ExecutionPlanMetricsSet::new(), - required_input_ordering: aggr_exec.required_input_ordering()[0].clone(), - limit: aggr_exec.limit().clone(), - input_order_mode: aggr_exec.input_order_mode().clone(), - cache: aggr_exec.cache().clone(), - }) as Arc - } else { - plan - }; - - println!("start optimize"); - self.optimize_internal(plan, session_state, |_, _| {}) } } From d20dde38f85fcfdee198f530b3d01e41541892c2 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 10:56:46 +0800 Subject: [PATCH 03/77] revert test result Signed-off-by: jayzhan211 --- datafusion/sqllogictest/test_files/aggregate.slt | 13 +++++++------ datafusion/sqllogictest/test_files/explain.slt | 3 +++ datafusion/sqllogictest/test_files/join.slt | 2 +- datafusion/sqllogictest/test_files/predicates.slt | 2 +- datafusion/sqllogictest/test_files/timestamps.slt | 2 +- 5 files changed, 13 insertions(+), 9 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 5827ad77ad76..4929ab485d6d 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -123,7 +123,7 @@ LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; query ? select array_agg(c1 order by c2 desc, c3) from agg_order; ---- -[1, 2, 3, 4, 5, 6, 7, 8, 9, 10] +[5, 6, 7, 8, 9, 1, 2, 3, 4, 10] query TT explain select array_agg(c1 order by c2 desc, c3) from agg_order; @@ -135,8 +135,9 @@ physical_plan AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type statement ok @@ -151,8 +152,8 @@ CREATE TABLE array_agg_order_list_table AS VALUES query T? rowsort select column1, array_agg(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [[4, 5, 6], [7, 8, 9]] -w [[1, 2, 3], [9, 5, 2], [3, 2, 5]] +b [[7, 8, 9], [4, 5, 6]] +w [[3, 2, 5], [9, 5, 2], [1, 2, 3]] query T?? rowsort select column1, first_value(column3 order by column2, column4 desc), last_value(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; @@ -163,7 +164,7 @@ w [3, 2, 5] [1, 2, 3] query T? rowsort select column1, nth_value(column3, 2 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [7, 8, 9] +b [4, 5, 6] w [9, 5, 2] statement ok diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index b7ad36dace16..57ee8c311f6c 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -254,6 +254,7 @@ physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPAC physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -312,6 +313,7 @@ GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Co physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -348,6 +350,7 @@ GlobalLimitExec: skip=0, fetch=10 physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index da9b4168e7e0..135ab8075425 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -587,7 +587,7 @@ FROM t1 ---- 11 11 11 -# subsequent inner join +# subsequent inner join query III rowsort SELECT t1.t1_id, t2.t2_id, t3.t3_id FROM t1 diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 33c9ff7c3eed..4c9254beef6b 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -781,4 +781,4 @@ logical_plan EmptyRelation physical_plan EmptyExec statement ok -drop table t; \ No newline at end of file +drop table t; diff --git a/datafusion/sqllogictest/test_files/timestamps.slt b/datafusion/sqllogictest/test_files/timestamps.slt index f0e04b522a78..491b9b810687 100644 --- a/datafusion/sqllogictest/test_files/timestamps.slt +++ b/datafusion/sqllogictest/test_files/timestamps.slt @@ -2794,4 +2794,4 @@ SELECT '2000-12-01 04:04:12' AT TIME ZONE 'America/New York'; # abbreviated timezone is not supported statement error -SELECT '2023-03-12 02:00:00' AT TIME ZONE 'EDT'; \ No newline at end of file +SELECT '2023-03-12 02:00:00' AT TIME ZONE 'EDT'; From c671033424490ce69b07d1027d97d2d3d37e11fa Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 14:53:01 +0800 Subject: [PATCH 04/77] pass mulit order test Signed-off-by: jayzhan211 --- .../core/src/physical_optimizer/optimizer.rs | 7 +- .../physical_optimizer/output_requirements.rs | 9 + .../physical_optimizer/simplify_ordering.rs | 302 ++++++++----- datafusion/core/src/physical_planner.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 400 ++---------------- .../physical-plan/src/coalesce_partitions.rs | 8 + .../sqllogictest/test_files/aggregate.slt | 26 +- 7 files changed, 290 insertions(+), 466 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 931e20fff953..b777933e120e 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -76,10 +76,12 @@ impl PhysicalOptimizer { /// Create a new optimizer using the recommended list of rules pub fn new() -> Self { let rules: Vec> = vec![ + Arc::new(SimplifyOrdering::new()), // If there is a output requirement of the query, make sure that // this information is not lost across different rules during optimization. Arc::new(OutputRequirements::new_add_mode()), Arc::new(AggregateStatistics::new()), + Arc::new(SimplifyOrdering::new()), // Statistics-based join selection will change the Auto mode to a real join implementation, // like collect left, or hash join, or future sort merge join, which will influence the // EnforceDistribution and EnforceSorting rules as they decide whether to add additional @@ -90,13 +92,16 @@ impl PhysicalOptimizer { // as that rule may inject other operations in between the different AggregateExecs. // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), + Arc::new(SimplifyOrdering::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at // least one of the operators in the plan benefits from increased parallelism. Arc::new(EnforceDistribution::new()), + Arc::new(SimplifyOrdering::new()), // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule Arc::new(CombinePartialFinalAggregate::new()), + Arc::new(SimplifyOrdering::new()), // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule @@ -120,6 +125,7 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), + Arc::new(SimplifyOrdering::new()), // The ProjectionPushdown rule tries to push projections towards // the sources in the execution plan. As a result of this process, // a projection can disappear if it reaches the source providers, and @@ -127,7 +133,6 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), - Arc::new(SimplifyOrdering::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 829d523c990c..386340bbeda6 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -94,6 +94,15 @@ pub(crate) struct OutputRequirementExec { } impl OutputRequirementExec { + pub fn clone_with_input(&self, input: Arc) -> Self { + Self { + input, + order_requirement: self.order_requirement.clone(), + dist_requirement: self.dist_requirement.clone(), + cache: self.cache.clone(), + } + } + pub(crate) fn new( input: Arc, requirements: Option, diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index fd0dd746abd7..bb02df69b92b 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -17,10 +17,11 @@ use std::sync::Arc; +use arrow::compute::kernels::aggregate; use datafusion_common::{ config::ConfigOptions, not_impl_err, - tree_node::{Transformed, TransformedResult, TreeNode}, + tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, }; use datafusion_physical_expr::{ aggregate::is_order_sensitive, @@ -30,13 +31,16 @@ use datafusion_physical_expr::{ }; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, - metrics::ExecutionPlanMetricsSet, + coalesce_partitions::CoalescePartitionsExec, ExecutionPlan, ExecutionPlanProperties, }; +use datafusion_physical_expr::equivalence::collapse_lex_req; +use datafusion_physical_plan::windows::get_ordered_partition_by_indices; + use crate::error::Result; -use super::PhysicalOptimizerRule; +use super::{output_requirements::OutputRequirementExec, PhysicalOptimizerRule}; #[derive(Default)] pub struct SimplifyOrdering {} @@ -53,8 +57,12 @@ impl PhysicalOptimizerRule for SimplifyOrdering { plan: Arc, _config: &ConfigOptions, ) -> Result> { - plan.transform_down(&get_common_requirement_of_aggregate_input) - .data() + let res = plan + .transform_down(&get_common_requirement_of_aggregate_input) + .data(); + + // println!("res: {:?}", res); + res } fn name(&self) -> &str { @@ -69,104 +77,100 @@ impl PhysicalOptimizerRule for SimplifyOrdering { fn get_common_requirement_of_aggregate_input( plan: Arc, ) -> Result>> { - let mut is_transformed = false; + let children = plan.children(); + + let new_c: Option>> = if children.is_empty() { + None + } else { + assert_eq!(children.len(), 1, "AggregateExec should have one child"); + let c = children[0].clone(); + + // for c in children { + // let new_c = get_common_requirement_of_aggregate_input(c.clone())?; + // if new_c.transformed { + // is_transformed = true; + // } + // } - let plan = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let input = if let Some(aggr_exec) = - aggr_exec.input().as_any().downcast_ref::() + let new_c = get_common_requirement_of_aggregate_input(c)?; + Some(new_c) + }; + + let plan = optimize_internal(plan)?; + // println!("t: {} plan: {:?}", plan.transformed, plan); + + if let Some(c) = new_c { + if !c.transformed { + return Ok(plan); + } + + let plan = plan.data; + + // TODO: support more types of ExecutionPlan + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let p = aggr_exec.clone_with_input(c.data); + return Ok(Transformed::yes(Arc::new(p) as Arc)); + } else if let Some(coalesce_exec) = + plan.as_any().downcast_ref::() { - // println!("AggregateExec"); - let input = aggr_exec.input().clone(); - let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); - let group_by = aggr_exec.group_by(); - let mode = aggr_exec.mode().clone(); - - use datafusion_physical_expr::equivalence::collapse_lex_req; - use datafusion_physical_plan::windows::get_ordered_partition_by_indices; - - let input_eq_properties = input.equivalence_properties(); - let groupby_exprs = group_by.input_exprs(); - let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let mut new_requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - - // print!("my new_requirement: {:?}", new_requirement); - // println!("my aggr_expr: {:?}", aggr_expr); - // println!("my group_by: {:?}", group_by); - // println!("my input_eq_properties: {:?}", input_eq_properties); - // println!("my mode: {:?}", mode); - let req = get_aggregate_exprs_requirement( - &new_requirement, - &mut aggr_expr, - &group_by, - input_eq_properties, - &mode, - )?; - // println!("my req: {:?}", req); - new_requirement.extend(req); - new_requirement = collapse_lex_req(new_requirement); - let required_input_ordering = - (!new_requirement.is_empty()).then_some(new_requirement); - - // println!( - // "my required_input_ordering: {:?}", - // required_input_ordering - // ); - let filter_expr = aggr_exec.filter_expr().to_vec(); - - // let metrics = aggr_exec.metrics(); - - let p = AggregateExec { - mode, - group_by: group_by.clone(), - aggr_expr, - filter_expr, - input, - schema: aggr_exec.schema().clone(), - input_schema: aggr_exec.input_schema().clone(), - metrics: ExecutionPlanMetricsSet::new(), - required_input_ordering, - limit: None, - input_order_mode: aggr_exec.input_order_mode().clone(), - cache: aggr_exec.cache().clone(), - }; - - is_transformed = true; - - Arc::new(p) as Arc + let p = coalesce_exec.clone_with_input(c.data); + return Ok(Transformed::yes(Arc::new(p) as Arc)); + } else if let Some(out_req_exec) = + plan.as_any().downcast_ref::() + { + let p = out_req_exec.clone_with_input(c.data); + return Ok(Transformed::yes(Arc::new(p) as Arc)); } else { - aggr_exec.input().clone() - }; + return not_impl_err!("Unsupported ExecutionPlan type: {}", plan.name()); + } + } - // TODO: modify the input of aggr_exec - - // aggr_exec as Arc - - Arc::new(AggregateExec { - mode: aggr_exec.mode().clone(), - group_by: aggr_exec.group_by().clone(), - aggr_expr: aggr_exec.aggr_expr().to_vec(), - filter_expr: aggr_exec.filter_expr().to_vec(), - input, - schema: aggr_exec.schema().clone(), - input_schema: aggr_exec.input_schema().clone(), - metrics: ExecutionPlanMetricsSet::new(), - required_input_ordering: aggr_exec.required_input_ordering()[0].clone(), - limit: aggr_exec.limit().clone(), - input_order_mode: aggr_exec.input_order_mode().clone(), - cache: aggr_exec.cache().clone(), - }) as Arc - } else { - plan - }; + return Ok(plan); +} + +fn optimize_internal( + plan: Arc, +) -> Result>> { + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + if aggr_exec.mode() != &AggregateMode::Partial { + return Ok(Transformed::no(plan)); + } + + let input = aggr_exec.input().clone(); + let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); + let group_by = aggr_exec.group_by(); + let mode = aggr_exec.mode(); + + let input_eq_properties = input.equivalence_properties(); + let groupby_exprs = group_by.input_exprs(); + // If existing ordering satisfies a prefix of the GROUP BY expressions, + // prefix requirements with this section. In this case, aggregation will + // work more efficiently. + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let mut new_requirement = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + + let req = get_aggregate_exprs_requirement( + &new_requirement, + &mut aggr_expr, + &group_by, + input_eq_properties, + mode, + )?; + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); + + let p = aggr_exec.clone_with_required_input_ordering(required_input_ordering); - if is_transformed { - Ok(Transformed::yes(plan)) + let res = Arc::new(p) as Arc; + Ok(Transformed::yes(res)) } else { Ok(Transformed::no(plan)) } @@ -389,3 +393,105 @@ fn get_aggregate_exprs_requirement( } Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } + + +mod tests { + use super::*; + use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; + use datafusion_physical_expr::{expressions::{col, OrderSensitiveArrayAgg}, PhysicalSortExpr}; + + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); + + Ok(schema) + } + + #[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 col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema); + // Columns a and b are equal. + eq_properties.add_equal_conditions(col_a, col_b); + // Aggregate requirements are + // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively + let order_by_exprs = vec![ + None, + Some(vec![PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options: options1, + }, + ]), + ]; + let common_requirement = vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]; + let mut aggr_exprs = order_by_exprs + .into_iter() + .map(|order_by_expr| { + Arc::new(OrderSensitiveArrayAgg::new( + col_a.clone(), + "array_agg", + DataType::Int32, + false, + vec![], + order_by_expr.unwrap_or_default(), + )) as _ + }) + .collect::>(); + let group_by = PhysicalGroupBy::new_single(vec![]); + let res = get_aggregate_exprs_requirement( + &[], + &mut aggr_exprs, + &group_by, + &eq_properties, + &AggregateMode::Partial, + )?; + let res = PhysicalSortRequirement::to_sort_exprs(res); + assert_eq!(res, common_requirement); + Ok(()) + } +} \ No newline at end of file diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 798dfa2d5bfb..0159b460f310 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -465,7 +465,9 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; - self.optimize_internal(plan, session_state, |_, _| {}) + let res = self.optimize_internal(plan, session_state, |_, _| {}); + println!("optimized done"); + res } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 63ba87ee4434..4f0cc4495ecb 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -36,15 +36,13 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::{internal_err, not_impl_err, Result}; +use datafusion_common::{internal_err, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ - aggregate::is_order_sensitive, - equivalence::{collapse_lex_req, ProjectionMapping}, - expressions::{Column, FirstValue, LastValue, Max, Min, UnKnownColumn}, - physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, - LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortRequirement, + equivalence::ProjectionMapping, + expressions::{Column, Max, Min, UnKnownColumn}, + AggregateExpr, LexRequirement, PhysicalExpr, }; use itertools::Itertools; @@ -269,13 +267,42 @@ pub struct AggregateExec { } impl AggregateExec { - pub fn with_required_input_ordering( - self, + pub fn clone_with_input(&self, input: Arc) -> Self { + Self { + input, + // clone the rest of the fields + mode: self.mode, + group_by: self.group_by.clone(), + aggr_expr: self.aggr_expr.clone(), + filter_expr: self.filter_expr.clone(), + limit: self.limit, + schema: self.schema.clone(), + input_schema: self.input_schema.clone(), + metrics: self.metrics.clone(), + input_order_mode: self.input_order_mode.clone(), + cache: self.cache.clone(), + required_input_ordering: self.required_input_ordering.clone(), + } + } + + pub fn clone_with_required_input_ordering( + &self, required_input_ordering: Option, ) -> Self { Self { required_input_ordering, - ..self + // clone the rest of the fields + mode: self.mode, + group_by: self.group_by.clone(), + aggr_expr: self.aggr_expr.clone(), + filter_expr: self.filter_expr.clone(), + limit: self.limit, + input: self.input.clone(), + schema: self.schema.clone(), + input_schema: self.input_schema.clone(), + metrics: self.metrics.clone(), + input_order_mode: self.input_order_mode.clone(), + cache: self.cache.clone(), } } @@ -324,7 +351,7 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - mut aggr_expr: Vec>, + aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -335,35 +362,13 @@ impl AggregateExec { return internal_err!("Inconsistent aggregate expr: {:?} and filter expr: {:?} for AggregateExec, their size should match", aggr_expr, filter_expr); } - let input_eq_properties = input.equivalence_properties(); // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of the GROUP BY expressions, // prefix requirements with this section. In this case, aggregation will // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let mut new_requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - - println!("correct new_requirement: {:?}", new_requirement); - println!("correct aggr_expr: {:?}", aggr_expr); - println!("correct group_by: {:?}", group_by); - println!("correct input_eq_properties: {:?}", input_eq_properties); - println!("correct mode: {:?}", mode); - let req = get_aggregate_exprs_requirement( - &new_requirement, - &mut aggr_expr, - &group_by, - input_eq_properties, - &mode, - )?; - new_requirement.extend(req); - new_requirement = collapse_lex_req(new_requirement); + let copied_aggr_expr = aggr_expr.clone(); let input_order_mode = if indices.len() == groupby_exprs.len() && !indices.is_empty() { @@ -378,16 +383,6 @@ impl AggregateExec { let projection_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; - let required_input_ordering = - (!new_requirement.is_empty()).then_some(new_requirement); - - if required_input_ordering.is_some() { - println!( - "correct required_input_ordering: {:?}", - required_input_ordering - ); - } - let required_input_ordering = None; let cache = Self::compute_properties( @@ -400,7 +395,7 @@ impl AggregateExec { Ok(AggregateExec { mode, group_by, - aggr_expr, + aggr_expr: copied_aggr_expr, filter_expr, input, schema, @@ -823,224 +818,6 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } -/// Determines the lexical ordering requirement for an aggregate expression. -/// -/// # Parameters -/// -/// - `aggr_expr`: A reference to an `Arc` representing the -/// aggregate expression. -/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the -/// physical GROUP BY expression. -/// - `agg_mode`: A reference to an `AggregateMode` instance representing the -/// mode of aggregation. -/// -/// # Returns -/// -/// A `LexOrdering` instance indicating the lexical ordering requirement for -/// the aggregate expression. -fn get_aggregate_expr_req( - aggr_expr: &Arc, - group_by: &PhysicalGroupBy, - agg_mode: &AggregateMode, -) -> LexOrdering { - // If the aggregation function is not order sensitive, or the aggregation - // is performing a "second stage" calculation, or all aggregate function - // requirements are inside the GROUP BY expression, then ignore the ordering - // requirement. - if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { - return vec![]; - } - - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); - - // In non-first stage modes, we accumulate data (using `merge_batch`) from - // different partitions (i.e. merge partial results). During this merge, we - // consider the ordering of each partial result. Hence, we do not need to - // use the ordering requirement in such modes as long as partial results are - // generated with the correct ordering. - if group_by.is_single() { - // Remove all orderings that occur in the group by. These requirements - // will definitely be satisfied -- Each group by expression will have - // distinct values per group, hence all requirements are satisfied. - let physical_exprs = group_by.input_exprs(); - req.retain(|sort_expr| { - !physical_exprs_contains(&physical_exprs, &sort_expr.expr) - }); - } - req -} - -/// Computes the finer ordering for between given existing ordering requirement -/// of aggregate expression. -/// -/// # Parameters -/// -/// * `existing_req` - The existing lexical ordering that needs refinement. -/// * `aggr_expr` - A reference to an aggregate expression trait object. -/// * `group_by` - Information about the physical grouping (e.g group by expression). -/// * `eq_properties` - Equivalence properties relevant to the computation. -/// * `agg_mode` - The mode of aggregation (e.g., Partial, Final, etc.). -/// -/// # Returns -/// -/// An `Option` representing the computed finer lexical ordering, -/// or `None` if there is no finer ordering; e.g. the existing requirement and -/// the aggregator requirement is incompatible. -fn finer_ordering( - existing_req: &LexOrdering, - aggr_expr: &Arc, - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Option { - let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - eq_properties.get_finer_ordering(existing_req, &aggr_req) -} - -/// Concatenates the given slices. -fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { - [lhs, rhs].concat() -} - -/// Get the common requirement that satisfies all the aggregate expressions. -/// -/// # Parameters -/// -/// - `aggr_exprs`: A slice of `Arc` containing all the -/// aggregate expressions. -/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the -/// physical GROUP BY expression. -/// - `eq_properties`: A reference to an `EquivalenceProperties` instance -/// representing equivalence properties for ordering. -/// - `agg_mode`: A reference to an `AggregateMode` instance representing the -/// mode of aggregation. -/// -/// # Returns -/// -/// A `LexRequirement` instance, which is the requirement that satisfies all the -/// aggregate requirements. Returns an error in case of conflicting requirements. -fn get_aggregate_exprs_requirement( - prefix_requirement: &[PhysicalSortRequirement], - aggr_exprs: &mut [Arc], - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Result { - let mut requirement = vec![]; - for aggr_expr in aggr_exprs.iter_mut() { - let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_req = reverse_order_bys(aggr_req); - let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - - if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { - let mut first_value = first_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to LAST_VALUE enables more efficient execution - // given the existing ordering: - let mut last_value = first_value.convert_to_last(); - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; - } - continue; - } - if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { - let mut last_value = last_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to FIRST_VALUE enables more efficient execution - // given the existing ordering: - let mut first_value = last_value.convert_to_first(); - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - last_value = last_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(last_value) as _; - } - continue; - } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Requirement is satisfied by existing ordering - requirement = finer_ordering; - continue; - } - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Reverse requirement is satisfied by exiting ordering. - // Hence reverse the aggregator - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } - } - } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - // There is a requirement that both satisfies existing requirement and current - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - continue; - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - // There is a requirement that both satisfies existing requirement and reverse - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } - } - // Neither the existing requirement and current aggregate requirement satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); - } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) -} - /// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: /// * Partial: AggregateExpr::expressions @@ -1263,27 +1040,15 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, LastValue, Median, OrderSensitiveArrayAgg, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg }; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, - PhysicalSortExpr, + PhysicalSortExpr, PhysicalSortRequirement, }; use futures::{FutureExt, Stream}; - // Generate a schema which consists of 5 columns (a, b, c, d, e) - fn create_test_schema() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); - - Ok(schema) - } - /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. @@ -2127,89 +1892,6 @@ 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 col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let mut eq_properties = EquivalenceProperties::new(test_schema); - // Columns a and b are equal. - eq_properties.add_equal_conditions(col_a, col_b); - // Aggregate requirements are - // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively - let order_by_exprs = vec![ - None, - Some(vec![PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }]), - Some(vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_b.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_c.clone(), - options: options1, - }, - ]), - Some(vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_b.clone(), - options: options1, - }, - ]), - ]; - let common_requirement = vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_c.clone(), - options: options1, - }, - ]; - let mut aggr_exprs = order_by_exprs - .into_iter() - .map(|order_by_expr| { - Arc::new(OrderSensitiveArrayAgg::new( - col_a.clone(), - "array_agg", - DataType::Int32, - false, - vec![], - order_by_expr.unwrap_or_default(), - )) as _ - }) - .collect::>(); - let group_by = PhysicalGroupBy::new_single(vec![]); - let res = get_aggregate_exprs_requirement( - &[], - &mut aggr_exprs, - &group_by, - &eq_properties, - &AggregateMode::Partial, - )?; - let res = PhysicalSortRequirement::to_sort_exprs(res); - assert_eq!(res, common_requirement); - Ok(()) - } #[test] fn test_agg_exec_same_schema() -> Result<()> { diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 1c725ce31f14..b95d17b627b9 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -72,6 +72,14 @@ impl CoalescePartitionsExec { input.execution_mode(), // Execution Mode ) } + + pub fn clone_with_input(&self, input: Arc) -> Self { + CoalescePartitionsExec { + input, + metrics: self.metrics.clone(), + cache: self.cache.clone(), + } + } } impl DisplayAs for CoalescePartitionsExec { diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 4929ab485d6d..135cded2e94f 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -123,7 +123,7 @@ LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; query ? select array_agg(c1 order by c2 desc, c3) from agg_order; ---- -[5, 6, 7, 8, 9, 1, 2, 3, 4, 10] +[1, 2, 3, 4, 5, 6, 7, 8, 9, 10] query TT explain select array_agg(c1 order by c2 desc, c3) from agg_order; @@ -135,9 +135,21 @@ physical_plan AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] -------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] ---------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true + +query TT +explain select array_agg(c1 order by c2 desc, c3) from agg_order; +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] +--TableScan: agg_order projection=[c1, c2, c3] +physical_plan +AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type statement ok @@ -152,8 +164,8 @@ CREATE TABLE array_agg_order_list_table AS VALUES query T? rowsort select column1, array_agg(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [[7, 8, 9], [4, 5, 6]] -w [[3, 2, 5], [9, 5, 2], [1, 2, 3]] +b [[4, 5, 6], [7, 8, 9]] +w [[1, 2, 3], [9, 5, 2], [3, 2, 5]] query T?? rowsort select column1, first_value(column3 order by column2, column4 desc), last_value(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; @@ -164,7 +176,7 @@ w [3, 2, 5] [1, 2, 3] query T? rowsort select column1, nth_value(column3, 2 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [4, 5, 6] +b [7, 8, 9] w [9, 5, 2] statement ok From 3f43dc19d1de68e323e216e50c18ce77a6c26eea Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 14:59:57 +0800 Subject: [PATCH 05/77] cleanup Signed-off-by: jayzhan211 --- datafusion/core/src/physical_optimizer/simplify_ordering.rs | 2 +- datafusion/core/src/physical_planner.rs | 4 +--- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index bb02df69b92b..67c964168787 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -82,7 +82,7 @@ fn get_common_requirement_of_aggregate_input( let new_c: Option>> = if children.is_empty() { None } else { - assert_eq!(children.len(), 1, "AggregateExec should have one child"); + assert_eq!(children.len(), 1, "children: {:?}", children); let c = children[0].clone(); // for c in children { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 0159b460f310..798dfa2d5bfb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -465,9 +465,7 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; - let res = self.optimize_internal(plan, session_state, |_, _| {}); - println!("optimized done"); - res + self.optimize_internal(plan, session_state, |_, _| {}) } } } From c8b46a49d4a0cc7a5e74ec89733e20e5f7234473 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 15:40:45 +0800 Subject: [PATCH 06/77] with new childes Signed-off-by: jayzhan211 --- .../core/src/physical_optimizer/optimizer.rs | 8 ++ .../physical_optimizer/simplify_ordering.rs | 85 +++++++++---------- .../physical-plan/src/aggregates/mod.rs | 4 +- 3 files changed, 53 insertions(+), 44 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index b777933e120e..e7fe76d42969 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -76,11 +76,13 @@ impl PhysicalOptimizer { /// Create a new optimizer using the recommended list of rules pub fn new() -> Self { let rules: Vec> = vec![ + // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), // If there is a output requirement of the query, make sure that // this information is not lost across different rules during optimization. Arc::new(OutputRequirements::new_add_mode()), Arc::new(AggregateStatistics::new()), + // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), // Statistics-based join selection will change the Auto mode to a real join implementation, // like collect left, or hash join, or future sort merge join, which will influence the @@ -92,15 +94,18 @@ impl PhysicalOptimizer { // as that rule may inject other operations in between the different AggregateExecs. // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), + // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at // least one of the operators in the plan benefits from increased parallelism. Arc::new(EnforceDistribution::new()), + // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule Arc::new(CombinePartialFinalAggregate::new()), + // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. @@ -115,6 +120,8 @@ impl PhysicalOptimizer { // Remove the ancillary output requirement operator since we are done with the planning // phase. Arc::new(OutputRequirements::new_remove_mode()), + // Appears after AggregateExec is created + Arc::new(SimplifyOrdering::new()), // The PipelineChecker rule will reject non-runnable query plans that use // pipeline-breaking operators on infinite input(s). The rule generates a // diagnostic error message when this happens. It makes no changes to the @@ -125,6 +132,7 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), + // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), // The ProjectionPushdown rule tries to push projections towards // the sources in the execution plan. As a result of this process, diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index 67c964168787..5f7faa92b789 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -79,53 +79,52 @@ fn get_common_requirement_of_aggregate_input( ) -> Result>> { let children = plan.children(); - let new_c: Option>> = if children.is_empty() { - None + let mut is_transformed = false; + let mut new_children: Vec> = vec![]; + for c in children.iter() { + let res = get_common_requirement_of_aggregate_input(c.clone())?; + if res.transformed { + is_transformed = true; + } + new_children.push(res.data); + } + + let plan = if is_transformed { + plan.with_new_children(new_children)? } else { - assert_eq!(children.len(), 1, "children: {:?}", children); - let c = children[0].clone(); - - // for c in children { - // let new_c = get_common_requirement_of_aggregate_input(c.clone())?; - // if new_c.transformed { - // is_transformed = true; - // } - // } - - let new_c = get_common_requirement_of_aggregate_input(c)?; - Some(new_c) + plan }; let plan = optimize_internal(plan)?; - // println!("t: {} plan: {:?}", plan.transformed, plan); - - if let Some(c) = new_c { - if !c.transformed { - return Ok(plan); - } - - let plan = plan.data; - - // TODO: support more types of ExecutionPlan - if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let p = aggr_exec.clone_with_input(c.data); - return Ok(Transformed::yes(Arc::new(p) as Arc)); - } else if let Some(coalesce_exec) = - plan.as_any().downcast_ref::() - { - let p = coalesce_exec.clone_with_input(c.data); - return Ok(Transformed::yes(Arc::new(p) as Arc)); - } else if let Some(out_req_exec) = - plan.as_any().downcast_ref::() - { - let p = out_req_exec.clone_with_input(c.data); - return Ok(Transformed::yes(Arc::new(p) as Arc)); - } else { - return not_impl_err!("Unsupported ExecutionPlan type: {}", plan.name()); - } - } - - return Ok(plan); + Ok(plan) + + // if let Some(c) = new_c { + // if !c.transformed { + // return Ok(plan); + // } + + // let plan = plan.data; + + // // TODO: support more types of ExecutionPlan + // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.clone_with_input(c.data); + // return Ok(Transformed::yes(Arc::new(p) as Arc)); + // } else if let Some(coalesce_exec) = + // plan.as_any().downcast_ref::() + // { + // let p = coalesce_exec.clone_with_input(c.data); + // return Ok(Transformed::yes(Arc::new(p) as Arc)); + // } else if let Some(out_req_exec) = + // plan.as_any().downcast_ref::() + // { + // let p = out_req_exec.clone_with_input(c.data); + // return Ok(Transformed::yes(Arc::new(p) as Arc)); + // } else { + // return not_impl_err!("Unsupported ExecutionPlan type: {}", plan.name()); + // } + // } + + // return Ok(plan); } fn optimize_internal( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4f0cc4495ecb..30fd28b538e8 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -698,6 +698,9 @@ impl ExecutionPlan for AggregateExec { self: Arc, children: Vec>, ) -> Result> { + // let aggr_exec = self.clone_with_input(children[0].clone()); + // Ok(Arc::new(aggr_exec)) + let mut me = AggregateExec::try_new_with_schema( self.mode, self.group_by.clone(), @@ -706,7 +709,6 @@ impl ExecutionPlan for AggregateExec { children[0].clone(), self.input_schema.clone(), self.schema.clone(), - //self.original_schema.clone(), )?; me.limit = self.limit; Ok(Arc::new(me)) From fcb297b7c9943d0d85b5996ec0eec1efc1e2c524 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 16:09:34 +0800 Subject: [PATCH 07/77] revert slt Signed-off-by: jayzhan211 --- .../sqllogictest/test_files/aggregate.slt | 28 ++++++------------- 1 file changed, 8 insertions(+), 20 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 135cded2e94f..b3a58dd582e7 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -123,7 +123,7 @@ LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; query ? select array_agg(c1 order by c2 desc, c3) from agg_order; ---- -[1, 2, 3, 4, 5, 6, 7, 8, 9, 10] +[5, 6, 7, 8, 9, 1, 2, 3, 4, 10] query TT explain select array_agg(c1 order by c2 desc, c3) from agg_order; @@ -135,21 +135,9 @@ physical_plan AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true - -query TT -explain select array_agg(c1 order by c2 desc, c3) from agg_order; ----- -logical_plan -Aggregate: groupBy=[[]], aggr=[[ARRAY_AGG(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]]] ---TableScan: agg_order projection=[c1, c2, c3] -physical_plan -AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(agg_order.c1)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true +------SortExec: expr=[c2@1 DESC,c3@2 ASC NULLS LAST] +--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type statement ok @@ -164,8 +152,8 @@ CREATE TABLE array_agg_order_list_table AS VALUES query T? rowsort select column1, array_agg(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [[4, 5, 6], [7, 8, 9]] -w [[1, 2, 3], [9, 5, 2], [3, 2, 5]] +b [[7, 8, 9], [4, 5, 6]] +w [[3, 2, 5], [9, 5, 2], [1, 2, 3]] query T?? rowsort select column1, first_value(column3 order by column2, column4 desc), last_value(column3 order by column2, column4 desc) from array_agg_order_list_table group by column1; @@ -176,7 +164,7 @@ w [3, 2, 5] [1, 2, 3] query T? rowsort select column1, nth_value(column3, 2 order by column2, column4 desc) from array_agg_order_list_table group by column1; ---- -b [7, 8, 9] +b [4, 5, 6] w [9, 5, 2] statement ok @@ -3439,4 +3427,4 @@ SELECT LAST_VALUE(column1 ORDER BY column2 DESC) IGNORE NULLS FROM t; 3 statement ok -DROP TABLE t; +DROP TABLE t; \ No newline at end of file From ab10168bf050b549bea9a0b3823a3f8b759161ec Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 16:24:44 +0800 Subject: [PATCH 08/77] revert back Signed-off-by: jayzhan211 --- .../physical_optimizer/simplify_ordering.rs | 11 +- .../physical-plan/src/aggregates/mod.rs | 303 +++++++++++++++++- 2 files changed, 302 insertions(+), 12 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index 5f7faa92b789..e6ddcdc25720 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -57,12 +57,13 @@ impl PhysicalOptimizerRule for SimplifyOrdering { plan: Arc, _config: &ConfigOptions, ) -> Result> { - let res = plan - .transform_down(&get_common_requirement_of_aggregate_input) - .data(); + Ok(plan) + // let res = plan + // .transform_down(&get_common_requirement_of_aggregate_input) + // .data(); - // println!("res: {:?}", res); - res + // // println!("res: {:?}", res); + // res } fn name(&self) -> &str { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 30fd28b538e8..c9ade05711c7 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -36,9 +36,13 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::{internal_err, Result}; +use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; +use datafusion_physical_expr::aggregate::is_order_sensitive; +use datafusion_physical_expr::equivalence::collapse_lex_req; +use datafusion_physical_expr::expressions::{FirstValue, LastValue}; +use datafusion_physical_expr::{physical_exprs_contains, reverse_order_bys, EquivalenceProperties, LexOrdering, PhysicalSortRequirement}; use datafusion_physical_expr::{ equivalence::ProjectionMapping, expressions::{Column, Max, Min, UnKnownColumn}, @@ -328,7 +332,7 @@ impl AggregateExec { )?; let schema = Arc::new(schema); - AggregateExec::try_new_with_schema( + let res = AggregateExec::try_new_with_schema( mode, group_by, aggr_expr, @@ -336,7 +340,50 @@ impl AggregateExec { input, input_schema, schema, - ) + )?; + + res.rewrite_ordering() + } + + fn rewrite_ordering(mut self) -> Result { + let input = self.input(); + let group_by = self.group_by(); + let mut aggr_expr = self.aggr_expr().to_vec(); + let mode = self.mode(); + + let input_eq_properties = input.equivalence_properties(); + // Get GROUP BY expressions: + let groupby_exprs = group_by.input_exprs(); + // If existing ordering satisfies a prefix of the GROUP BY expressions, + // prefix requirements with this section. In this case, aggregation will + // work more efficiently. + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + + let mut new_requirement = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + + let req = get_aggregate_exprs_requirement( + &new_requirement, + &mut aggr_expr, + &group_by, + input_eq_properties, + &mode, + )?; + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); + + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); + + self.aggr_expr = aggr_expr; + self.required_input_ordering = required_input_ordering; + + Ok(self) } /// Create a new hash aggregate execution plan with the given schema. @@ -351,7 +398,7 @@ impl AggregateExec { fn try_new_with_schema( mode: AggregateMode, group_by: PhysicalGroupBy, - aggr_expr: Vec>, + mut aggr_expr: Vec>, filter_expr: Vec>>, input: Arc, input_schema: SchemaRef, @@ -362,13 +409,30 @@ impl AggregateExec { return internal_err!("Inconsistent aggregate expr: {:?} and filter expr: {:?} for AggregateExec, their size should match", aggr_expr, filter_expr); } + let input_eq_properties = input.equivalence_properties(); // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of the GROUP BY expressions, // prefix requirements with this section. In this case, aggregation will // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let copied_aggr_expr = aggr_expr.clone(); + let mut new_requirement = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + + let req = get_aggregate_exprs_requirement( + &new_requirement, + &mut aggr_expr, + &group_by, + input_eq_properties, + &mode, + )?; + new_requirement.extend(req); + new_requirement = collapse_lex_req(new_requirement); let input_order_mode = if indices.len() == groupby_exprs.len() && !indices.is_empty() { @@ -383,7 +447,8 @@ impl AggregateExec { let projection_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; - let required_input_ordering = None; + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); let cache = Self::compute_properties( &input, @@ -392,10 +457,12 @@ impl AggregateExec { &mode, &input_order_mode, ); + + let required_input_ordering = None; Ok(AggregateExec { mode, group_by, - aggr_expr: copied_aggr_expr, + aggr_expr, filter_expr, input, schema, @@ -408,6 +475,7 @@ impl AggregateExec { }) } + /// Aggregation mode (full, partial) pub fn mode(&self) -> &AggregateMode { &self.mode @@ -711,6 +779,9 @@ impl ExecutionPlan for AggregateExec { self.schema.clone(), )?; me.limit = self.limit; + + me = me.rewrite_ordering()?; + Ok(Arc::new(me)) } @@ -820,6 +891,224 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } +/// Determines the lexical ordering requirement for an aggregate expression. +/// +/// # Parameters +/// +/// - `aggr_expr`: A reference to an `Arc` representing the +/// aggregate expression. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexOrdering` instance indicating the lexical ordering requirement for +/// the aggregate expression. +fn get_aggregate_expr_req( + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + agg_mode: &AggregateMode, +) -> LexOrdering { + // If the aggregation function is not order sensitive, or the aggregation + // is performing a "second stage" calculation, or all aggregate function + // requirements are inside the GROUP BY expression, then ignore the ordering + // requirement. + if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { + return vec![]; + } + + let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + + // In non-first stage modes, we accumulate data (using `merge_batch`) from + // different partitions (i.e. merge partial results). During this merge, we + // consider the ordering of each partial result. Hence, we do not need to + // use the ordering requirement in such modes as long as partial results are + // generated with the correct ordering. + if group_by.is_single() { + // Remove all orderings that occur in the group by. These requirements + // will definitely be satisfied -- Each group by expression will have + // distinct values per group, hence all requirements are satisfied. + let physical_exprs = group_by.input_exprs(); + req.retain(|sort_expr| { + !physical_exprs_contains(&physical_exprs, &sort_expr.expr) + }); + } + req +} + +/// Computes the finer ordering for between given existing ordering requirement +/// of aggregate expression. +/// +/// # Parameters +/// +/// * `existing_req` - The existing lexical ordering that needs refinement. +/// * `aggr_expr` - A reference to an aggregate expression trait object. +/// * `group_by` - Information about the physical grouping (e.g group by expression). +/// * `eq_properties` - Equivalence properties relevant to the computation. +/// * `agg_mode` - The mode of aggregation (e.g., Partial, Final, etc.). +/// +/// # Returns +/// +/// An `Option` representing the computed finer lexical ordering, +/// or `None` if there is no finer ordering; e.g. the existing requirement and +/// the aggregator requirement is incompatible. +fn finer_ordering( + existing_req: &LexOrdering, + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Option { + let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); + eq_properties.get_finer_ordering(existing_req, &aggr_req) +} + +/// Concatenates the given slices. +fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { + [lhs, rhs].concat() +} + +/// Get the common requirement that satisfies all the aggregate expressions. +/// +/// # Parameters +/// +/// - `aggr_exprs`: A slice of `Arc` containing all the +/// aggregate expressions. +/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the +/// physical GROUP BY expression. +/// - `eq_properties`: A reference to an `EquivalenceProperties` instance +/// representing equivalence properties for ordering. +/// - `agg_mode`: A reference to an `AggregateMode` instance representing the +/// mode of aggregation. +/// +/// # Returns +/// +/// A `LexRequirement` instance, which is the requirement that satisfies all the +/// aggregate requirements. Returns an error in case of conflicting requirements. +fn get_aggregate_exprs_requirement( + prefix_requirement: &[PhysicalSortRequirement], + aggr_exprs: &mut [Arc], + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Result { + let mut requirement = vec![]; + for aggr_expr in aggr_exprs.iter_mut() { + let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); + let reverse_aggr_req = reverse_order_bys(aggr_req); + let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); + let reverse_aggr_req = + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + + if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { + let mut first_value = first_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to LAST_VALUE enables more efficient execution + // given the existing ordering: + let mut last_value = first_value.convert_to_last(); + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + first_value = first_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(first_value) as _; + } + continue; + } + if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { + let mut last_value = last_value.clone(); + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + last_value = last_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(last_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to FIRST_VALUE enables more efficient execution + // given the existing ordering: + let mut first_value = last_value.convert_to_first(); + first_value = first_value.with_requirement_satisfied(true); + *aggr_expr = Arc::new(first_value) as _; + } else { + // Requirement is not satisfied with existing ordering. + last_value = last_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(last_value) as _; + } + continue; + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Requirement is satisfied by existing ordering + requirement = finer_ordering; + continue; + } + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Reverse requirement is satisfied by exiting ordering. + // Hence reverse the aggregator + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + // There is a requirement that both satisfies existing requirement and current + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + continue; + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + // There is a requirement that both satisfies existing requirement and reverse + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + // Neither the existing requirement and current aggregate requirement satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); + } + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) +} + /// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: /// * Partial: AggregateExpr::expressions From fffb54a88403b58e8f33875fbf840c1f25857742 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 17:07:47 +0800 Subject: [PATCH 09/77] rm rewrite in new child Signed-off-by: jayzhan211 --- .../enforce_distribution.rs | 9 ++++++- .../physical_optimizer/output_requirements.rs | 13 +++++++++- .../physical-plan/src/aggregates/mod.rs | 25 +++++++++++++++---- datafusion/physical-plan/src/lib.rs | 12 ++++++++- .../physical-plan/src/recursive_query.rs | 12 ++++++++- 5 files changed, 62 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 145f08af76dd..81e7c2ba6c85 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1217,7 +1217,14 @@ fn ensure_distribution( // Data Arc::new(InterleaveExec::try_new(children_plans)?) } else { - plan.with_new_children(children_plans)? + let plan = plan.with_new_children(children_plans)?; + + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let p = aggr_exec.rewrite_ordering()?; + Arc::new(p) + } else { + plan + } }; Ok(Transformed::yes(DistributionContext::new( diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 386340bbeda6..0f7e86b38747 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -32,6 +32,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; +use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; @@ -283,7 +284,17 @@ fn require_top_ordering_helper( // be responsible for (i.e. the originator of) the global ordering. let (new_child, is_changed) = require_top_ordering_helper(children.swap_remove(0))?; - Ok((plan.with_new_children(vec![new_child])?, is_changed)) + + let plan = plan.with_new_children(vec![new_child])?; + + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let p = aggr_exec.rewrite_ordering()?; + Ok((Arc::new(p) as _, is_changed)) + } else { + Ok((plan, is_changed)) + } + + // Ok((plan.with_new_children(vec![new_child])?, is_changed)) } else { // Stop searching, there is no global ordering desired for the query. Ok((plan, false)) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c9ade05711c7..1f0919e3ba73 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -345,7 +345,7 @@ impl AggregateExec { res.rewrite_ordering() } - fn rewrite_ordering(mut self) -> Result { + pub fn rewrite_ordering(&self) -> Result { let input = self.input(); let group_by = self.group_by(); let mut aggr_expr = self.aggr_expr().to_vec(); @@ -380,10 +380,25 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - self.aggr_expr = aggr_expr; - self.required_input_ordering = required_input_ordering; + Ok(Self { + mode: self.mode().clone(), + group_by: self.group_by().clone(), + aggr_expr, + filter_expr: self.filter_expr().to_vec(), + input: self.input().clone(), + schema: self.schema.clone(), + input_schema: self.input_schema.clone(), + metrics: self.metrics.clone(), + required_input_ordering, + limit: self.limit, + input_order_mode: self.input_order_mode.clone(), + cache: self.cache.clone(), + }) + + // self.aggr_expr = aggr_expr; + // self.required_input_ordering = required_input_ordering; - Ok(self) + // Ok(()) } /// Create a new hash aggregate execution plan with the given schema. @@ -780,7 +795,7 @@ impl ExecutionPlan for AggregateExec { )?; me.limit = self.limit; - me = me.rewrite_ordering()?; + // let me = me.rewrite_ordering()?; Ok(Arc::new(me)) } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index e1c8489655bf..4407e8aa2cd6 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -27,6 +27,7 @@ use crate::metrics::MetricsSet; use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; +use aggregates::AggregateExec; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::config::ConfigOptions; @@ -673,7 +674,16 @@ pub fn with_new_children_if_necessary( .zip(old_children.iter()) .any(|(c1, c2)| !Arc::data_ptr_eq(c1, c2)) { - plan.with_new_children(children) + let plan = plan.with_new_children(children)?; + + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let p = aggr_exec.rewrite_ordering()?; + Ok(Arc::new(p)) + } else { + Ok(plan) + } + + } else { Ok(plan) } diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index ba7d1a54548a..f8b8c6b5e1c9 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -26,6 +26,7 @@ use super::{ work_table::{ReservedBatches, WorkTable, WorkTableExec}, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; +use crate::aggregates::AggregateExec; use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan}; use arrow::datatypes::SchemaRef; @@ -359,7 +360,16 @@ fn reset_plan_states(plan: Arc) -> Result() { + let p = aggr_exec.rewrite_ordering()?; + Ok(Transformed::yes(Arc::new(p))) + } else { + Ok(Transformed::yes(new_plan)) + } + + // Ok(Transformed::yes(new_plan)) } }) .data() From fec3eaf4012a31cdf06c849939afa76b2d23b42f Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 18:16:13 +0800 Subject: [PATCH 10/77] backup Signed-off-by: jayzhan211 --- .../enforce_distribution.rs | 13 ++++--- .../core/src/physical_optimizer/optimizer.rs | 2 + .../physical_optimizer/output_requirements.rs | 13 ++++--- .../replace_with_order_preserving_variants.rs | 23 +++++++++++ .../physical_optimizer/simplify_ordering.rs | 38 ++++++++++++++----- datafusion/physical-plan/src/lib.rs | 13 ++++--- datafusion/physical-plan/src/tree_node.rs | 13 ++++++- .../sqllogictest/test_files/aggregate.slt | 2 +- .../sqllogictest/test_files/explain.slt | 32 ++++++++++++++-- .../sqllogictest/test_files/group_by.slt | 3 ++ 10 files changed, 120 insertions(+), 32 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 81e7c2ba6c85..bb1704a56c2b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1219,12 +1219,13 @@ fn ensure_distribution( } else { let plan = plan.with_new_children(children_plans)?; - if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let p = aggr_exec.rewrite_ordering()?; - Arc::new(p) - } else { - plan - } + // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.rewrite_ordering()?; + // Arc::new(p) + // } else { + // plan + // } + plan }; Ok(Transformed::yes(DistributionContext::new( diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index e7fe76d42969..92393940cc63 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -81,6 +81,7 @@ impl PhysicalOptimizer { // If there is a output requirement of the query, make sure that // this information is not lost across different rules during optimization. Arc::new(OutputRequirements::new_add_mode()), + Arc::new(SimplifyOrdering::new()), Arc::new(AggregateStatistics::new()), // Appears after AggregateExec is created Arc::new(SimplifyOrdering::new()), @@ -112,6 +113,7 @@ impl PhysicalOptimizer { // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. Arc::new(EnforceSorting::new()), + Arc::new(SimplifyOrdering::new()), // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. Arc::new(ProjectionPushdown::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 0f7e86b38747..62588629f0a6 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -287,12 +287,13 @@ fn require_top_ordering_helper( let plan = plan.with_new_children(vec![new_child])?; - if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let p = aggr_exec.rewrite_ordering()?; - Ok((Arc::new(p) as _, is_changed)) - } else { - Ok((plan, is_changed)) - } + // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.rewrite_ordering()?; + // Ok((Arc::new(p) as _, is_changed)) + // } else { + // } + + Ok((plan, is_changed)) // Ok((plan.with_new_children(vec![new_child])?, is_changed)) } else { diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index ad19215fbf67..7479dec4efbc 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -29,6 +29,7 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; +use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; @@ -138,6 +139,18 @@ fn plan_with_order_preserving_variants( } } + // let mut plan_context = sort_input.update_plan_from_children()?; + // let plan = plan_context.plan.clone(); + // let p = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.rewrite_ordering()?; + // Arc::new(p) + // } else { + // plan + // }; + // plan_context.plan = p; + + // Ok(plan_context) + sort_input.update_plan_from_children() } @@ -184,6 +197,16 @@ fn plan_with_order_breaking_variants( let coalesce = CoalescePartitionsExec::new(child); sort_input.plan = Arc::new(coalesce) as _; } else { + // let mut pc = sort_input.update_plan_from_children()?; + // let plan = pc.plan.clone(); + // let p = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.rewrite_ordering()?; + // Arc::new(p) + // } else { + // plan + // }; + // pc.plan = p; + // return Ok(pc); return sort_input.update_plan_from_children(); } diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index e6ddcdc25720..6d76d4ff4d02 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -57,13 +57,13 @@ impl PhysicalOptimizerRule for SimplifyOrdering { plan: Arc, _config: &ConfigOptions, ) -> Result> { - Ok(plan) - // let res = plan - // .transform_down(&get_common_requirement_of_aggregate_input) - // .data(); + // Ok(plan) + let res = plan + .transform_down(&get_common_requirement_of_aggregate_input) + .data(); - // // println!("res: {:?}", res); - // res + // println!("res: {:?}", res); + res } fn name(&self) -> &str { @@ -90,13 +90,33 @@ fn get_common_requirement_of_aggregate_input( new_children.push(res.data); } + let mode1 = plan.properties().execution_mode(); + // println!("mode 1: {:?}", mode); + let plan = if is_transformed { + + // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.clone_with_input(new_children[0].clone()); + // Arc::new(p) as Arc + // } else { + // plan.with_new_children(new_children)? + // } plan.with_new_children(new_children)? + } else { plan }; + let mode2 = plan.properties().execution_mode(); + if mode1 != mode2 { + println!("mode1: {:?}, mode2: {:?}", mode1, mode2); + } + let plan = optimize_internal(plan)?; + let mode3 = plan.data.properties().execution_mode(); + if mode1 != mode3 { + println!("mode1: {:?}, mode3: {:?}", mode1, mode3); + } Ok(plan) // if let Some(c) = new_c { @@ -132,9 +152,9 @@ fn optimize_internal( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - if aggr_exec.mode() != &AggregateMode::Partial { - return Ok(Transformed::no(plan)); - } + // if aggr_exec.mode() != &AggregateMode::Partial { + // return Ok(Transformed::no(plan)); + // } let input = aggr_exec.input().clone(); let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 4407e8aa2cd6..3f25855230b0 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -676,12 +676,13 @@ pub fn with_new_children_if_necessary( { let plan = plan.with_new_children(children)?; - if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let p = aggr_exec.rewrite_ordering()?; - Ok(Arc::new(p)) - } else { - Ok(plan) - } + // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.rewrite_ordering()?; + // Ok(Arc::new(p)) + // } else { + // Ok(plan) + // } + Ok(plan) } else { diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs index 52a52f81bdaf..435aa625710b 100644 --- a/datafusion/physical-plan/src/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -20,6 +20,7 @@ use std::fmt::{self, Display, Formatter}; use std::sync::Arc; +use crate::aggregates::AggregateExec; use crate::{displayable, with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{ConcreteTreeNode, DynTreeNode}; @@ -63,7 +64,17 @@ impl PlanContext { pub fn update_plan_from_children(mut self) -> Result { let children_plans = self.children.iter().map(|c| c.plan.clone()).collect(); - self.plan = with_new_children_if_necessary(self.plan, children_plans)?; + let plan = with_new_children_if_necessary(self.plan, children_plans)?; + + let plan = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + let p = aggr_exec.rewrite_ordering()?; + Arc::new(p) + } else { + plan + }; + + self.plan = plan; + Ok(self) } } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index b3a58dd582e7..4929ab485d6d 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3427,4 +3427,4 @@ SELECT LAST_VALUE(column1 ORDER BY column2 DESC) IGNORE NULLS FROM t; 3 statement ok -DROP TABLE t; \ No newline at end of file +DROP TABLE t; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 57ee8c311f6c..848ba4096359 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -239,22 +239,30 @@ logical_plan after optimize_projections SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan after SimpleOrdering CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after OutputRequirements OutputRequirementExec --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -295,25 +303,33 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; initial_physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after OutputRequirements OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -332,25 +348,35 @@ GlobalLimitExec: skip=0, fetch=10 initial_physical_plan_with_stats GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after SimpleOrdering +GlobalLimitExec: skip=0, fetch=10 +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after OutputRequirements OutputRequirementExec --GlobalLimitExec: skip=0, fetch=10 ----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after aggregate_statistics SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 869462b4722a..748a02aa0afe 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4249,6 +4249,9 @@ SELECT date_bin('15 minutes', ts) as time_chunks 2018-12-13T12:00:00 2018-11-13T17:00:00 + + + # Since extract is not a monotonic function, below query should not run. # when source is unbounded. query error From 5b68b482b20778ef0046c7772549faf1288d6194 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Wed, 10 Apr 2024 22:07:56 +0800 Subject: [PATCH 11/77] only move conversion to optimizer Signed-off-by: jayzhan211 --- .../enforce_distribution.rs | 15 ++- .../core/src/physical_optimizer/optimizer.rs | 21 ++-- .../physical_optimizer/output_requirements.rs | 12 +-- .../replace_with_order_preserving_variants.rs | 22 ---- .../physical_optimizer/simplify_ordering.rs | 6 -- .../physical-plan/src/aggregates/mod.rs | 100 ++++-------------- datafusion/physical-plan/src/lib.rs | 12 +-- .../physical-plan/src/recursive_query.rs | 11 +- datafusion/physical-plan/src/tree_node.rs | 9 +- .../sqllogictest/test_files/explain.slt | 29 ----- .../sqllogictest/test_files/group_by.slt | 15 +-- 11 files changed, 50 insertions(+), 202 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index bb1704a56c2b..fe43890bc850 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1080,6 +1080,11 @@ fn ensure_distribution( } }; + // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // let p = aggr_exec.rewrite_ordering()?; + // plan = Arc::new(p); + // } + // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. // - Satisfy the distribution requirements of every child, if it is not @@ -1217,15 +1222,7 @@ fn ensure_distribution( // Data Arc::new(InterleaveExec::try_new(children_plans)?) } else { - let plan = plan.with_new_children(children_plans)?; - - // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.rewrite_ordering()?; - // Arc::new(p) - // } else { - // plan - // } - plan + plan.with_new_children(children_plans)? }; Ok(Transformed::yes(DistributionContext::new( diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 92393940cc63..28d7e9b17fed 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -77,14 +77,14 @@ impl PhysicalOptimizer { pub fn new() -> Self { let rules: Vec> = vec![ // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // If there is a output requirement of the query, make sure that // this information is not lost across different rules during optimization. Arc::new(OutputRequirements::new_add_mode()), - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), Arc::new(AggregateStatistics::new()), // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // Statistics-based join selection will change the Auto mode to a real join implementation, // like collect left, or hash join, or future sort merge join, which will influence the // EnforceDistribution and EnforceSorting rules as they decide whether to add additional @@ -96,24 +96,24 @@ impl PhysicalOptimizer { // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at // least one of the operators in the plan benefits from increased parallelism. Arc::new(EnforceDistribution::new()), // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule Arc::new(CombinePartialFinalAggregate::new()), // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. Arc::new(EnforceSorting::new()), - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. Arc::new(ProjectionPushdown::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the @@ -123,7 +123,7 @@ impl PhysicalOptimizer { // phase. Arc::new(OutputRequirements::new_remove_mode()), // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // The PipelineChecker rule will reject non-runnable query plans that use // pipeline-breaking operators on infinite input(s). The rule generates a // diagnostic error message when this happens. It makes no changes to the @@ -134,8 +134,7 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), - // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), + // Arc::new(SimplifyOrdering::new()), // The ProjectionPushdown rule tries to push projections towards // the sources in the execution plan. As a result of this process, // a projection can disappear if it reaches the source providers, and @@ -143,6 +142,8 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), + // Appears after AggregateExec is created + // Arc::new(SimplifyOrdering::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 62588629f0a6..0d2a888ee2d2 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -285,17 +285,7 @@ fn require_top_ordering_helper( let (new_child, is_changed) = require_top_ordering_helper(children.swap_remove(0))?; - let plan = plan.with_new_children(vec![new_child])?; - - // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.rewrite_ordering()?; - // Ok((Arc::new(p) as _, is_changed)) - // } else { - // } - - Ok((plan, is_changed)) - - // Ok((plan.with_new_children(vec![new_child])?, is_changed)) + Ok((plan.with_new_children(vec![new_child])?, is_changed)) } else { // Stop searching, there is no global ordering desired for the query. Ok((plan, false)) diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 7479dec4efbc..c486d13caa17 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -139,18 +139,6 @@ fn plan_with_order_preserving_variants( } } - // let mut plan_context = sort_input.update_plan_from_children()?; - // let plan = plan_context.plan.clone(); - // let p = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.rewrite_ordering()?; - // Arc::new(p) - // } else { - // plan - // }; - // plan_context.plan = p; - - // Ok(plan_context) - sort_input.update_plan_from_children() } @@ -197,16 +185,6 @@ fn plan_with_order_breaking_variants( let coalesce = CoalescePartitionsExec::new(child); sort_input.plan = Arc::new(coalesce) as _; } else { - // let mut pc = sort_input.update_plan_from_children()?; - // let plan = pc.plan.clone(); - // let p = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.rewrite_ordering()?; - // Arc::new(p) - // } else { - // plan - // }; - // pc.plan = p; - // return Ok(pc); return sort_input.update_plan_from_children(); } diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index 6d76d4ff4d02..8375e825c633 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -95,12 +95,6 @@ fn get_common_requirement_of_aggregate_input( let plan = if is_transformed { - // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.clone_with_input(new_children[0].clone()); - // Arc::new(p) as Arc - // } else { - // plan.with_new_children(new_children)? - // } plan.with_new_children(new_children)? } else { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 1f0919e3ba73..fcc1400c21f2 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -332,7 +332,7 @@ impl AggregateExec { )?; let schema = Arc::new(schema); - let res = AggregateExec::try_new_with_schema( + AggregateExec::try_new_with_schema( mode, group_by, aggr_expr, @@ -340,65 +340,7 @@ impl AggregateExec { input, input_schema, schema, - )?; - - res.rewrite_ordering() - } - - pub fn rewrite_ordering(&self) -> Result { - let input = self.input(); - let group_by = self.group_by(); - let mut aggr_expr = self.aggr_expr().to_vec(); - let mode = self.mode(); - - let input_eq_properties = input.equivalence_properties(); - // Get GROUP BY expressions: - let groupby_exprs = group_by.input_exprs(); - // If existing ordering satisfies a prefix of the GROUP BY expressions, - // prefix requirements with this section. In this case, aggregation will - // work more efficiently. - let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - - let mut new_requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - - let req = get_aggregate_exprs_requirement( - &new_requirement, - &mut aggr_expr, - &group_by, - input_eq_properties, - &mode, - )?; - new_requirement.extend(req); - new_requirement = collapse_lex_req(new_requirement); - - let required_input_ordering = - (!new_requirement.is_empty()).then_some(new_requirement); - - Ok(Self { - mode: self.mode().clone(), - group_by: self.group_by().clone(), - aggr_expr, - filter_expr: self.filter_expr().to_vec(), - input: self.input().clone(), - schema: self.schema.clone(), - input_schema: self.input_schema.clone(), - metrics: self.metrics.clone(), - required_input_ordering, - limit: self.limit, - input_order_mode: self.input_order_mode.clone(), - cache: self.cache.clone(), - }) - - // self.aggr_expr = aggr_expr; - // self.required_input_ordering = required_input_ordering; - - // Ok(()) + ) } /// Create a new hash aggregate execution plan with the given schema. @@ -473,7 +415,7 @@ impl AggregateExec { &input_order_mode, ); - let required_input_ordering = None; + // let required_input_ordering = None; Ok(AggregateExec { mode, group_by, @@ -1025,15 +967,15 @@ fn get_aggregate_exprs_requirement( )) { first_value = first_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(first_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to LAST_VALUE enables more efficient execution - // given the existing ordering: - let mut last_value = first_value.convert_to_last(); - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; + // } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + // prefix_requirement, + // &reverse_aggr_req, + // )) { + // // Converting to LAST_VALUE enables more efficient execution + // // given the existing ordering: + // let mut last_value = first_value.convert_to_last(); + // last_value = last_value.with_requirement_satisfied(true); + // *aggr_expr = Arc::new(last_value) as _; } else { // Requirement is not satisfied with existing ordering. first_value = first_value.with_requirement_satisfied(false); @@ -1049,15 +991,15 @@ fn get_aggregate_exprs_requirement( )) { last_value = last_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(last_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to FIRST_VALUE enables more efficient execution - // given the existing ordering: - let mut first_value = last_value.convert_to_first(); - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; + // } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + // prefix_requirement, + // &reverse_aggr_req, + // )) { + // // Converting to FIRST_VALUE enables more efficient execution + // // given the existing ordering: + // let mut first_value = last_value.convert_to_first(); + // first_value = first_value.with_requirement_satisfied(true); + // *aggr_expr = Arc::new(first_value) as _; } else { // Requirement is not satisfied with existing ordering. last_value = last_value.with_requirement_satisfied(false); diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 3f25855230b0..8890777b40eb 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -674,17 +674,7 @@ pub fn with_new_children_if_necessary( .zip(old_children.iter()) .any(|(c1, c2)| !Arc::data_ptr_eq(c1, c2)) { - let plan = plan.with_new_children(children)?; - - // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.rewrite_ordering()?; - // Ok(Arc::new(p)) - // } else { - // Ok(plan) - // } - Ok(plan) - - + plan.with_new_children(children) } else { Ok(plan) } diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index f8b8c6b5e1c9..91abd5cfdf28 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -360,16 +360,7 @@ fn reset_plan_states(plan: Arc) -> Result() { - let p = aggr_exec.rewrite_ordering()?; - Ok(Transformed::yes(Arc::new(p))) - } else { - Ok(Transformed::yes(new_plan)) - } - - // Ok(Transformed::yes(new_plan)) + Ok(Transformed::yes(new_plan)) } }) .data() diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs index 435aa625710b..86cf2d055474 100644 --- a/datafusion/physical-plan/src/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -17,6 +17,7 @@ //! This module provides common traits for visiting or rewriting tree nodes easily. +use core::panic; use std::fmt::{self, Display, Formatter}; use std::sync::Arc; @@ -65,14 +66,6 @@ impl PlanContext { pub fn update_plan_from_children(mut self) -> Result { let children_plans = self.children.iter().map(|c| c.plan.clone()).collect(); let plan = with_new_children_if_necessary(self.plan, children_plans)?; - - let plan = if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let p = aggr_exec.rewrite_ordering()?; - Arc::new(p) - } else { - plan - }; - self.plan = plan; Ok(self) diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 848ba4096359..b7ad36dace16 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -239,29 +239,20 @@ logical_plan after optimize_projections SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -physical_plan after SimpleOrdering CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after OutputRequirements OutputRequirementExec --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after aggregate_statistics SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -303,32 +294,23 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; initial_physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after OutputRequirements OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] ----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after aggregate_statistics SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -348,34 +330,23 @@ GlobalLimitExec: skip=0, fetch=10 initial_physical_plan_with_stats GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after SimpleOrdering -GlobalLimitExec: skip=0, fetch=10 ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after OutputRequirements OutputRequirementExec --GlobalLimitExec: skip=0, fetch=10 ----ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after aggregate_statistics SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 748a02aa0afe..56b17a976dc3 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2677,7 +2677,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2708,7 +2708,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2740,7 +2740,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2805,7 +2805,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ------TableScan: sales_global projection=[country, ts, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] ----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -3157,7 +3157,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------------SortExec: expr=[amount@1 DESC] ----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3800,10 +3800,10 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -4252,6 +4252,7 @@ SELECT date_bin('15 minutes', ts) as time_chunks + # Since extract is not a monotonic function, below query should not run. # when source is unbounded. query error From 2ab74c94fa7f2d4dd5a0c11a50cad26ca5b06274 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 11 Apr 2024 09:34:03 +0800 Subject: [PATCH 12/77] find test that do reverse Signed-off-by: jayzhan211 --- .../core/src/physical_optimizer/optimizer.rs | 2 +- .../physical_optimizer/simplify_ordering.rs | 3 ++ .../physical-plan/src/aggregates/mod.rs | 24 ++++++++------ .../sqllogictest/test_files/explain.slt | 3 ++ datafusion/sqllogictest/test_files/test1.slt | 32 +++++++++++++++++++ 5 files changed, 54 insertions(+), 10 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/test1.slt diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 28d7e9b17fed..636d9a20e7a0 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -143,7 +143,7 @@ impl PhysicalOptimizer { // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), + Arc::new(SimplifyOrdering::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index 8375e825c633..8e07436215e1 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -176,6 +176,9 @@ fn optimize_internal( input_eq_properties, mode, )?; + + // println!("req: {:?}", req); + new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); let required_input_ordering = diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index fcc1400c21f2..3500ae4c10c5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -967,15 +967,21 @@ fn get_aggregate_exprs_requirement( )) { first_value = first_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(first_value) as _; - // } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - // prefix_requirement, - // &reverse_aggr_req, - // )) { - // // Converting to LAST_VALUE enables more efficient execution - // // given the existing ordering: - // let mut last_value = first_value.convert_to_last(); - // last_value = last_value.with_requirement_satisfied(true); - // *aggr_expr = Arc::new(last_value) as _; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to LAST_VALUE enables more efficient execution + // given the existing ordering: + // let mut last_value = first_value.convert_to_last(); + // last_value = last_value.with_requirement_satisfied(true); + // *aggr_expr = Arc::new(last_value) as _; + + + println!("can convertt {:?}", aggr_expr); + + first_value = first_value.with_requirement_satisfied(false); + *aggr_expr = Arc::new(first_value) as _; } else { // Requirement is not satisfied with existing ordering. first_value = first_value.with_requirement_satisfied(false); diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index b7ad36dace16..57ee8c311f6c 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -254,6 +254,7 @@ physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPAC physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -312,6 +313,7 @@ GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Co physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -348,6 +350,7 @@ GlobalLimitExec: skip=0, fetch=10 physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt new file mode 100644 index 000000000000..2b3775ff3a36 --- /dev/null +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -0,0 +1,32 @@ +# 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. + +statement ok +CREATE EXTERNAL TABLE agg_order ( +c1 INT NOT NULL, +c2 INT NOT NULL, +c3 INT NOT NULL +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (c1 ASC, c2 ASC) +WITH ORDER (c3 ASC) +LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; + +# test array_agg with order by multiple columns +query I +select first_value(c1 order by c3 desc) from agg_order; From 03abf1999dbf06d1fc43be6932f65ed28c11fa4e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 11 Apr 2024 20:20:02 +0800 Subject: [PATCH 13/77] add test for first and last Signed-off-by: jayzhan211 --- .../core/tests/data/convert_first_last.csv | 11 +++++++++++ datafusion/sqllogictest/test_files/test1.slt | 19 ++++++++++++++----- 2 files changed, 25 insertions(+), 5 deletions(-) create mode 100644 datafusion/core/tests/data/convert_first_last.csv diff --git a/datafusion/core/tests/data/convert_first_last.csv b/datafusion/core/tests/data/convert_first_last.csv new file mode 100644 index 000000000000..059b631e5711 --- /dev/null +++ b/datafusion/core/tests/data/convert_first_last.csv @@ -0,0 +1,11 @@ +c1,c2,c3 +1,9,0 +2,8,1 +3,7,2 +4,6,3 +5,5,4 +6,4,5 +7,3,6 +8,2,7 +9,1,8 +10,0,9 \ No newline at end of file diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt index 2b3775ff3a36..5fc11514e671 100644 --- a/datafusion/sqllogictest/test_files/test1.slt +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -16,17 +16,26 @@ # under the License. statement ok -CREATE EXTERNAL TABLE agg_order ( +CREATE EXTERNAL TABLE convert_first_last_table ( c1 INT NOT NULL, c2 INT NOT NULL, c3 INT NOT NULL ) STORED AS CSV WITH HEADER ROW -WITH ORDER (c1 ASC, c2 ASC) +WITH ORDER (c1 ASC) +WITH ORDER (c2 DESC) WITH ORDER (c3 ASC) -LOCATION '../core/tests/data/aggregate_agg_multi_order.csv'; +LOCATION '../core/tests/data/convert_first_last.csv'; -# test array_agg with order by multiple columns +# test first to last query I -select first_value(c1 order by c3 desc) from agg_order; +select first_value(c1 order by c3 desc) from convert_first_last_table; +---- +10 + +# test last to first +query I +select last_value(c1 order by c2 asc) from convert_first_last_table; +---- +1 From 83c721a5218af1f214add92f3dde606a5791797a Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 11 Apr 2024 22:24:21 +0800 Subject: [PATCH 14/77] pass all test Signed-off-by: jayzhan211 --- .../core/src/physical_optimizer/optimizer.rs | 2 +- .../physical_optimizer/simplify_ordering.rs | 144 +++++++++++++++--- datafusion/core/src/physical_planner.rs | 2 + .../src/equivalence/properties.rs | 3 + .../physical-plan/src/aggregates/mod.rs | 88 +++++------ .../physical-plan/src/coalesce_partitions.rs | 14 +- datafusion/sqllogictest/test_files/test1.slt | 22 ++- 7 files changed, 196 insertions(+), 79 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 636d9a20e7a0..494fd4d49002 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -76,6 +76,7 @@ impl PhysicalOptimizer { /// Create a new optimizer using the recommended list of rules pub fn new() -> Self { let rules: Vec> = vec![ + Arc::new(SimplifyOrdering::new()), // Appears after AggregateExec is created // Arc::new(SimplifyOrdering::new()), // If there is a output requirement of the query, make sure that @@ -143,7 +144,6 @@ impl PhysicalOptimizer { // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), // Appears after AggregateExec is created - Arc::new(SimplifyOrdering::new()), ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index 8e07436215e1..041399102eb2 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -15,9 +15,8 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - use arrow::compute::kernels::aggregate; +use datafusion_common::Result; use datafusion_common::{ config::ConfigOptions, not_impl_err, @@ -25,6 +24,7 @@ use datafusion_common::{ }; use datafusion_physical_expr::{ aggregate::is_order_sensitive, + equivalence::ProjectionMapping, expressions::{FirstValue, LastValue}, physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalSortRequirement, @@ -32,14 +32,13 @@ use datafusion_physical_expr::{ use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, coalesce_partitions::CoalescePartitionsExec, - ExecutionPlan, ExecutionPlanProperties, + ExecutionPlan, ExecutionPlanProperties, InputOrderMode, }; +use std::sync::Arc; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; -use crate::error::Result; - use super::{output_requirements::OutputRequirementExec, PhysicalOptimizerRule}; #[derive(Default)] @@ -80,12 +79,12 @@ fn get_common_requirement_of_aggregate_input( ) -> Result>> { let children = plan.children(); - let mut is_transformed = false; + let mut is_child_transformed = false; let mut new_children: Vec> = vec![]; for c in children.iter() { let res = get_common_requirement_of_aggregate_input(c.clone())?; if res.transformed { - is_transformed = true; + is_child_transformed = true; } new_children.push(res.data); } @@ -93,10 +92,12 @@ fn get_common_requirement_of_aggregate_input( let mode1 = plan.properties().execution_mode(); // println!("mode 1: {:?}", mode); - let plan = if is_transformed { + println!("plan name: {:?}", plan.name()); + println!("is_child_transformed: {:?}", is_child_transformed); + println!("new_children: {:?}", new_children); + let plan = if is_child_transformed { plan.with_new_children(new_children)? - } else { plan }; @@ -107,11 +108,35 @@ fn get_common_requirement_of_aggregate_input( } let plan = optimize_internal(plan)?; + + if plan.transformed { + println!("transformed plan: {:?}", plan.data.name()); + } else { + println!("not transformed plan: {:?}", plan.data.name()); + } + let mode3 = plan.data.properties().execution_mode(); if mode1 != mode3 { println!("mode1: {:?}, mode3: {:?}", mode1, mode3); } - Ok(plan) + + if !plan.transformed { + let name = plan.data.name(); + println!( + "not transformed plan: {:?} and is_child_transformed: {:?}", + name, is_child_transformed + ); + } + + // If one of the children is transformed, then the plan is considered transformed, then we update + // the children of the plan from bottom to top. + if plan.transformed || is_child_transformed { + Ok(Transformed::yes(plan.data)) + } else { + Ok(Transformed::no(plan.data)) + } + + // Ok(plan) // if let Some(c) = new_c { // if !c.transformed { @@ -142,25 +167,52 @@ fn get_common_requirement_of_aggregate_input( // return Ok(plan); } +fn try_get_updated_aggr_expr_from_child( + aggr_exec: &AggregateExec, +) -> Vec> { + let input = aggr_exec.input(); + if aggr_exec.mode() != &AggregateMode::Partial { + // Some aggregators may be modified during initialization for + // optimization purposes. For example, a FIRST_VALUE may turn + // into a LAST_VALUE with the reverse ordering requirement. + // To reflect such changes to subsequent stages, use the updated + // `AggregateExpr`/`PhysicalSortExpr` objects. + // + // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] + + if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { + if c_aggr_exec.mode() == &AggregateMode::Partial { + // If the input is an AggregateExec in Partial mode, then the + // input is a CoalescePartitionsExec. In this case, the + // AggregateExec is the second stage of aggregation. The + // requirements of the second stage are the requirements of + // the first stage. + return c_aggr_exec.aggr_expr().to_vec(); + } + } + } + + aggr_exec.aggr_expr().to_vec() +} + fn optimize_internal( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // if aggr_exec.mode() != &AggregateMode::Partial { - // return Ok(Transformed::no(plan)); - // } + println!("mode: {:?}", aggr_exec.mode()); - let input = aggr_exec.input().clone(); - let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); + let input = aggr_exec.input(); + let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); let group_by = aggr_exec.group_by(); let mode = aggr_exec.mode(); let input_eq_properties = input.equivalence_properties(); + // println!("input_eq_properties: {:?}", input_eq_properties); let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of the GROUP BY expressions, // prefix requirements with this section. In this case, aggregation will // work more efficiently. - let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let indices = get_ordered_partition_by_indices(&groupby_exprs, input); let mut new_requirement = indices .iter() .map(|&idx| PhysicalSortRequirement { @@ -169,6 +221,7 @@ fn optimize_internal( }) .collect::>(); + println!("1 aggr_expr: {:?}", aggr_expr); let req = get_aggregate_exprs_requirement( &new_requirement, &mut aggr_expr, @@ -176,6 +229,7 @@ fn optimize_internal( input_eq_properties, mode, )?; + println!("2 aggr_expr: {:?}", aggr_expr); // println!("req: {:?}", req); @@ -184,7 +238,48 @@ fn optimize_internal( let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let p = aggr_exec.clone_with_required_input_ordering(required_input_ordering); + println!("required_input_ordering: {:?}", required_input_ordering); + println!("agg_expr: {:?}", aggr_expr); + + let input_order_mode = + if indices.len() == groupby_exprs.len() && !indices.is_empty() { + InputOrderMode::Sorted + } else if !indices.is_empty() { + InputOrderMode::PartiallySorted(indices) + } else { + InputOrderMode::Linear + }; + let projection_mapping = + ProjectionMapping::try_new(&group_by.expr(), &input.schema())?; + + let cache = AggregateExec::compute_properties( + &input, + plan.schema().clone(), + &projection_mapping, + &mode, + &input_order_mode, + ); + + // let p = AggregateExec { + // mode: mode.clone(), + // group_by: group_by.clone(), + // aggr_expr, + // filter_expr: aggr_exec.filter_expr().to_vec(), + // input: input.clone(), + // schema: aggr_exec.schema(), + // input_schema: aggr_exec.input_schema(), + // metrics: aggr_exec.metrics().clone(), + // required_input_ordering, + // limit: None, + // input_order_mode, + // cache, + // }; + let p = aggr_exec.clone_with_required_input_ordering_and_aggr_expr( + required_input_ordering, + aggr_expr, + cache, + input_order_mode, + ); let res = Arc::new(p) as Arc; Ok(Transformed::yes(res)) @@ -306,16 +401,23 @@ fn get_aggregate_exprs_requirement( if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { let mut first_value = first_value.clone(); + + // println!("prefix_requirement: {:?}", prefix_requirement); + // println!("aggr_req: {:?}", aggr_req); + // println!("reverse_aggr_req: {:?}", reverse_aggr_req); + if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &aggr_req, )) { + println!("1st step"); first_value = first_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(first_value) as _; } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &reverse_aggr_req, )) { + println!("2nd step"); // Converting to LAST_VALUE enables more efficient execution // given the existing ordering: let mut last_value = first_value.convert_to_last(); @@ -411,11 +513,13 @@ fn get_aggregate_exprs_requirement( Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } - mod tests { use super::*; use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; - use datafusion_physical_expr::{expressions::{col, OrderSensitiveArrayAgg}, PhysicalSortExpr}; + use datafusion_physical_expr::{ + expressions::{col, OrderSensitiveArrayAgg}, + PhysicalSortExpr, + }; fn create_test_schema() -> Result { let a = Field::new("a", DataType::Int32, true); @@ -511,4 +615,4 @@ mod tests { assert_eq!(res, common_requirement); Ok(()) } -} \ No newline at end of file +} diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 798dfa2d5bfb..40a1a60d6865 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -465,6 +465,8 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; + // println!("first plan: {:#?}", plan); + self.optimize_internal(plan, session_state, |_, _| {}) } } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index c14c88d6c69b..1f434d0149a9 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -287,9 +287,11 @@ impl EquivalenceProperties { let mut eq_properties = self.clone(); // First, standardize the given requirement: let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); + // println!("normalized_reqs: {:?}", normalized_reqs); for normalized_req in normalized_reqs { // Check whether given ordering is satisfied if !eq_properties.ordering_satisfy_single(&normalized_req) { + // println!("not single"); return false; } // Treat satisfied keys as constants in subsequent iterations. We @@ -325,6 +327,7 @@ impl EquivalenceProperties { fn ordering_satisfy_single(&self, req: &PhysicalSortRequirement) -> bool { let expr_ordering = self.get_expr_ordering(req.expr.clone()); let ExprOrdering { expr, data, .. } = expr_ordering; + // println!("expr: {:?}, data: {:?}", expr, data); match data { SortProperties::Ordered(options) => { let sort_expr = PhysicalSortExpr { expr, options }; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3500ae4c10c5..d089e89c3cb1 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -42,12 +42,15 @@ use datafusion_expr::Accumulator; use datafusion_physical_expr::aggregate::is_order_sensitive; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::expressions::{FirstValue, LastValue}; -use datafusion_physical_expr::{physical_exprs_contains, reverse_order_bys, EquivalenceProperties, LexOrdering, PhysicalSortRequirement}; use datafusion_physical_expr::{ equivalence::ProjectionMapping, expressions::{Column, Max, Min, UnKnownColumn}, AggregateExpr, LexRequirement, PhysicalExpr, }; +use datafusion_physical_expr::{ + physical_exprs_contains, reverse_order_bys, EquivalenceProperties, LexOrdering, + PhysicalSortRequirement, +}; use itertools::Itertools; @@ -271,42 +274,45 @@ pub struct AggregateExec { } impl AggregateExec { - pub fn clone_with_input(&self, input: Arc) -> Self { - Self { - input, - // clone the rest of the fields - mode: self.mode, - group_by: self.group_by.clone(), - aggr_expr: self.aggr_expr.clone(), - filter_expr: self.filter_expr.clone(), - limit: self.limit, - schema: self.schema.clone(), - input_schema: self.input_schema.clone(), - metrics: self.metrics.clone(), - input_order_mode: self.input_order_mode.clone(), - cache: self.cache.clone(), - required_input_ordering: self.required_input_ordering.clone(), - } - } - - pub fn clone_with_required_input_ordering( + // pub fn clone_with_input(&self, input: Arc) -> Self { + // Self { + // input, + // // clone the rest of the fields + // mode: self.mode, + // group_by: self.group_by.clone(), + // aggr_expr: self.aggr_expr.clone(), + // filter_expr: self.filter_expr.clone(), + // limit: self.limit, + // schema: self.schema.clone(), + // input_schema: self.input_schema.clone(), + // metrics: self.metrics.clone(), + // input_order_mode: self.input_order_mode.clone(), + // cache: self.cache.clone(), + // required_input_ordering: self.required_input_ordering.clone(), + // } + // } + + pub fn clone_with_required_input_ordering_and_aggr_expr( &self, required_input_ordering: Option, + aggr_expr: Vec>, + cache: PlanProperties, + input_order_mode: InputOrderMode, ) -> Self { Self { required_input_ordering, // clone the rest of the fields mode: self.mode, group_by: self.group_by.clone(), - aggr_expr: self.aggr_expr.clone(), + aggr_expr, filter_expr: self.filter_expr.clone(), limit: self.limit, input: self.input.clone(), schema: self.schema.clone(), input_schema: self.input_schema.clone(), - metrics: self.metrics.clone(), - input_order_mode: self.input_order_mode.clone(), - cache: self.cache.clone(), + metrics: ExecutionPlanMetricsSet::new(), + input_order_mode, + cache, } } @@ -432,7 +438,6 @@ impl AggregateExec { }) } - /// Aggregation mode (full, partial) pub fn mode(&self) -> &AggregateMode { &self.mode @@ -555,7 +560,7 @@ impl AggregateExec { } /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( + pub fn compute_properties( input: &Arc, schema: SchemaRef, projection_mapping: &ProjectionMapping, @@ -960,6 +965,9 @@ fn get_aggregate_exprs_requirement( PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { + println!("prefix_requirement {:?}", prefix_requirement); + println!("reverse_aggr_req {:?}", reverse_aggr_req); + let mut first_value = first_value.clone(); if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, @@ -967,21 +975,15 @@ fn get_aggregate_exprs_requirement( )) { first_value = first_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(first_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to LAST_VALUE enables more efficient execution - // given the existing ordering: - // let mut last_value = first_value.convert_to_last(); - // last_value = last_value.with_requirement_satisfied(true); - // *aggr_expr = Arc::new(last_value) as _; - - - println!("can convertt {:?}", aggr_expr); - - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; + // } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + // prefix_requirement, + // &reverse_aggr_req, + // )) { + // // Converting to LAST_VALUE enables more efficient execution + // // given the existing ordering: + // let mut last_value = first_value.convert_to_last(); + // last_value = last_value.with_requirement_satisfied(true); + // *aggr_expr = Arc::new(last_value) as _; } else { // Requirement is not satisfied with existing ordering. first_value = first_value.with_requirement_satisfied(false); @@ -1001,6 +1003,7 @@ fn get_aggregate_exprs_requirement( // prefix_requirement, // &reverse_aggr_req, // )) { + // println!("last to first {:?}", aggr_expr); // // Converting to FIRST_VALUE enables more efficient execution // // given the existing ordering: // let mut first_value = last_value.convert_to_first(); @@ -1294,7 +1297,7 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, }; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, @@ -2146,7 +2149,6 @@ mod tests { Ok(()) } - #[test] fn test_agg_exec_same_schema() -> Result<()> { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index b95d17b627b9..67ceb07aeeb6 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -73,13 +73,13 @@ impl CoalescePartitionsExec { ) } - pub fn clone_with_input(&self, input: Arc) -> Self { - CoalescePartitionsExec { - input, - metrics: self.metrics.clone(), - cache: self.cache.clone(), - } - } + // pub fn clone_with_input(&self, input: Arc) -> Self { + // CoalescePartitionsExec { + // input, + // metrics: self.metrics.clone(), + // cache: self.cache.clone(), + // } + // } } impl DisplayAs for CoalescePartitionsExec { diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt index 5fc11514e671..0c903fdeb955 100644 --- a/datafusion/sqllogictest/test_files/test1.slt +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -28,14 +28,20 @@ WITH ORDER (c2 DESC) WITH ORDER (c3 ASC) LOCATION '../core/tests/data/convert_first_last.csv'; -# test first to last -query I -select first_value(c1 order by c3 desc) from convert_first_last_table; +# test first to last, the result does not show difference, we need to check the conversion by `explain` +query TT +explain select first_value(c1 order by c3 desc) from convert_first_last_table; ---- -10 +logical_plan +Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] +--TableScan: convert_first_last_table projection=[c1, c3] +physical_plan +AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true # test last to first -query I -select last_value(c1 order by c2 asc) from convert_first_last_table; ----- -1 +# query TT +# explain select last_value(c1 order by c2 asc) from convert_first_last_table; \ No newline at end of file From 34b8f01f057be7baf6dbea70b44ba5987fe91892 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Thu, 11 Apr 2024 22:27:15 +0800 Subject: [PATCH 15/77] upd test Signed-off-by: jayzhan211 --- .../core/src/physical_optimizer/simplify_ordering.rs | 2 +- datafusion/sqllogictest/test_files/explain.slt | 8 +++++--- datafusion/sqllogictest/test_files/group_by.slt | 6 +++--- datafusion/sqllogictest/test_files/test1.slt | 2 +- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index 041399102eb2..d62621c7af53 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -171,7 +171,7 @@ fn try_get_updated_aggr_expr_from_child( aggr_exec: &AggregateExec, ) -> Vec> { let input = aggr_exec.input(); - if aggr_exec.mode() != &AggregateMode::Partial { + if aggr_exec.mode() == &AggregateMode::Final || aggr_exec.mode() == &AggregateMode::FinalPartitioned { // Some aggregators may be modified during initialization for // optimization purposes. For example, a FIRST_VALUE may turn // into a LAST_VALUE with the reverse ordering requirement. diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 57ee8c311f6c..1e15f401be4d 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -239,6 +239,7 @@ logical_plan after optimize_projections SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] +physical_plan after SimpleOrdering CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after OutputRequirements OutputRequirementExec --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true @@ -254,7 +255,6 @@ physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPAC physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] @@ -295,6 +295,7 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; initial_physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after OutputRequirements OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -313,7 +314,6 @@ GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Co physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -332,6 +332,9 @@ GlobalLimitExec: skip=0, fetch=10 initial_physical_plan_with_stats GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] +physical_plan after SimpleOrdering +GlobalLimitExec: skip=0, fetch=10 +--ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after OutputRequirements OutputRequirementExec --GlobalLimitExec: skip=0, fetch=10 @@ -350,7 +353,6 @@ GlobalLimitExec: skip=0, fetch=10 physical_plan after PipelineChecker SAME TEXT AS ABOVE physical_plan after LimitAggregation SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan GlobalLimitExec: skip=0, fetch=10 --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 56b17a976dc3..8945f3b01c28 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2805,7 +2805,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ------TableScan: sales_global projection=[country, ts, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] ----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -3800,10 +3800,10 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt index 0c903fdeb955..4ca847fa01d8 100644 --- a/datafusion/sqllogictest/test_files/test1.slt +++ b/datafusion/sqllogictest/test_files/test1.slt @@ -44,4 +44,4 @@ AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) # test last to first # query TT -# explain select last_value(c1 order by c2 asc) from convert_first_last_table; \ No newline at end of file +# explain select last_value(c1 order by c2 asc) from convert_first_last_table; From b17e8f7bb0c839f6139fdee07e5dd6871d503159 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 08:57:15 +0800 Subject: [PATCH 16/77] upd test Signed-off-by: jayzhan211 --- .../enforce_distribution.rs | 5 --- .../core/src/physical_optimizer/optimizer.rs | 1 + .../physical_optimizer/output_requirements.rs | 9 ---- .../replace_with_order_preserving_variants.rs | 1 - .../physical_optimizer/simplify_ordering.rs | 12 +++--- datafusion/core/src/physical_planner.rs | 2 - .../src/equivalence/properties.rs | 2 - .../physical-plan/src/aggregates/mod.rs | 41 +++++++++++-------- .../sqllogictest/test_files/explain.slt | 3 ++ .../sqllogictest/test_files/group_by.slt | 8 ++-- 10 files changed, 37 insertions(+), 47 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index fe43890bc850..145f08af76dd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1080,11 +1080,6 @@ fn ensure_distribution( } }; - // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.rewrite_ordering()?; - // plan = Arc::new(p); - // } - // This loop iterates over all the children to: // - Increase parallelism for every child if it is beneficial. // - Satisfy the distribution requirements of every child, if it is not diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 494fd4d49002..e0263ced3890 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -114,6 +114,7 @@ impl PhysicalOptimizer { // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. Arc::new(EnforceSorting::new()), + Arc::new(SimplifyOrdering::new()), // Arc::new(SimplifyOrdering::new()), // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. Arc::new(ProjectionPushdown::new()), diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 0d2a888ee2d2..412976ca9e43 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -95,15 +95,6 @@ pub(crate) struct OutputRequirementExec { } impl OutputRequirementExec { - pub fn clone_with_input(&self, input: Arc) -> Self { - Self { - input, - order_requirement: self.order_requirement.clone(), - dist_requirement: self.dist_requirement.clone(), - cache: self.cache.clone(), - } - } - pub(crate) fn new( input: Arc, requirements: Option, diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index c486d13caa17..ad19215fbf67 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -29,7 +29,6 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; -use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index d62621c7af53..f70bae883409 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -199,7 +199,6 @@ fn optimize_internal( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - println!("mode: {:?}", aggr_exec.mode()); let input = aggr_exec.input(); let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); @@ -207,7 +206,6 @@ fn optimize_internal( let mode = aggr_exec.mode(); let input_eq_properties = input.equivalence_properties(); - // println!("input_eq_properties: {:?}", input_eq_properties); let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of the GROUP BY expressions, // prefix requirements with this section. In this case, aggregation will @@ -221,7 +219,6 @@ fn optimize_internal( }) .collect::>(); - println!("1 aggr_expr: {:?}", aggr_expr); let req = get_aggregate_exprs_requirement( &new_requirement, &mut aggr_expr, @@ -229,9 +226,6 @@ fn optimize_internal( input_eq_properties, mode, )?; - println!("2 aggr_expr: {:?}", aggr_expr); - - // println!("req: {:?}", req); new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); @@ -281,6 +275,12 @@ fn optimize_internal( input_order_mode, ); + // let aggr_exec = aggr_exec.to_owned().to_owned(); + // aggr_exec.with_aggr_expr(aggr_expr); + // aggr_exec.with_required_input_ordering(required_input_ordering); + // aggr_exec.with_cache(cache); + // aggr_exec.with_input_order_mode(input_order_mode); + let res = Arc::new(p) as Arc; Ok(Transformed::yes(res)) } else { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 40a1a60d6865..798dfa2d5bfb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -465,8 +465,6 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; - // println!("first plan: {:#?}", plan); - self.optimize_internal(plan, session_state, |_, _| {}) } } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 1f434d0149a9..367cc3543577 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -287,11 +287,9 @@ impl EquivalenceProperties { let mut eq_properties = self.clone(); // First, standardize the given requirement: let normalized_reqs = eq_properties.normalize_sort_requirements(reqs); - // println!("normalized_reqs: {:?}", normalized_reqs); for normalized_req in normalized_reqs { // Check whether given ordering is satisfied if !eq_properties.ordering_satisfy_single(&normalized_req) { - // println!("not single"); return false; } // Treat satisfied keys as constants in subsequent iterations. We diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index d089e89c3cb1..be4ba9fe8a16 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -274,24 +274,6 @@ pub struct AggregateExec { } impl AggregateExec { - // pub fn clone_with_input(&self, input: Arc) -> Self { - // Self { - // input, - // // clone the rest of the fields - // mode: self.mode, - // group_by: self.group_by.clone(), - // aggr_expr: self.aggr_expr.clone(), - // filter_expr: self.filter_expr.clone(), - // limit: self.limit, - // schema: self.schema.clone(), - // input_schema: self.input_schema.clone(), - // metrics: self.metrics.clone(), - // input_order_mode: self.input_order_mode.clone(), - // cache: self.cache.clone(), - // required_input_ordering: self.required_input_ordering.clone(), - // } - // } - pub fn clone_with_required_input_ordering_and_aggr_expr( &self, required_input_ordering: Option, @@ -438,6 +420,29 @@ impl AggregateExec { }) } + pub fn with_cache(mut self, cache: PlanProperties) -> Self { + self.cache = cache; + self + } + + pub fn with_input_order_mode(mut self, input_order_mode: InputOrderMode) -> Self { + self.input_order_mode = input_order_mode; + self + } + + pub fn with_required_input_ordering( + mut self, + required_input_ordering: Option, + ) -> Self { + self.required_input_ordering = required_input_ordering; + self + } + + pub fn with_aggr_expr(mut self, aggr_expr: Vec>) -> Self { + self.aggr_expr = aggr_expr; + self + } + /// Aggregation mode (full, partial) pub fn mode(&self) -> &AggregateMode { &self.mode diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 1e15f401be4d..2c82f2e6a68d 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -249,6 +249,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true @@ -306,6 +307,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements @@ -345,6 +347,7 @@ physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 8945f3b01c28..88e0272ef9bc 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2677,7 +2677,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2708,7 +2708,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2740,7 +2740,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ----TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] ----SortExec: expr=[amount@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -3157,7 +3157,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------------SortExec: expr=[amount@1 DESC] ----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------------MemoryExec: partitions=1, partition_sizes=[1] From a313af086357e0edc4ae83c0e9fb4039ea6c9ae4 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 09:28:54 +0800 Subject: [PATCH 17/77] cleanup Signed-off-by: jayzhan211 --- .../core/src/physical_optimizer/optimizer.rs | 24 ++--- .../physical_optimizer/output_requirements.rs | 1 - .../physical_optimizer/simplify_ordering.rs | 88 ++----------------- datafusion/core/src/physical_planner.rs | 2 + .../src/equivalence/properties.rs | 1 - .../physical-plan/src/aggregates/mod.rs | 15 ++-- .../sqllogictest/test_files/explain.slt | 8 +- 7 files changed, 24 insertions(+), 115 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index e0263ced3890..134d11e43a0e 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use super::projection_pushdown::ProjectionPushdown; -use super::simplify_ordering::SimplifyOrdering; +use super::simplify_ordering::ConvertFirstLast; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; @@ -76,16 +76,10 @@ impl PhysicalOptimizer { /// Create a new optimizer using the recommended list of rules pub fn new() -> Self { let rules: Vec> = vec![ - Arc::new(SimplifyOrdering::new()), - // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), // If there is a output requirement of the query, make sure that // this information is not lost across different rules during optimization. Arc::new(OutputRequirements::new_add_mode()), - // Arc::new(SimplifyOrdering::new()), Arc::new(AggregateStatistics::new()), - // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), // Statistics-based join selection will change the Auto mode to a real join implementation, // like collect left, or hash join, or future sort merge join, which will influence the // EnforceDistribution and EnforceSorting rules as they decide whether to add additional @@ -96,26 +90,22 @@ impl PhysicalOptimizer { // as that rule may inject other operations in between the different AggregateExecs. // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), - // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), + // Run once before PartialFinalAggregation is rewritten to ensure the rule is applied correctly + Arc::new(ConvertFirstLast::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at // least one of the operators in the plan benefits from increased parallelism. Arc::new(EnforceDistribution::new()), - // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), // The CombinePartialFinalAggregate rule should be applied after the EnforceDistribution rule Arc::new(CombinePartialFinalAggregate::new()), - // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), // The EnforceSorting rule is for adding essential local sorting to satisfy the required // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. Arc::new(EnforceSorting::new()), - Arc::new(SimplifyOrdering::new()), - // Arc::new(SimplifyOrdering::new()), + // Run once after the local sorting requirement is changed + Arc::new(ConvertFirstLast::new()), // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. Arc::new(ProjectionPushdown::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the @@ -124,8 +114,6 @@ impl PhysicalOptimizer { // Remove the ancillary output requirement operator since we are done with the planning // phase. Arc::new(OutputRequirements::new_remove_mode()), - // Appears after AggregateExec is created - // Arc::new(SimplifyOrdering::new()), // The PipelineChecker rule will reject non-runnable query plans that use // pipeline-breaking operators on infinite input(s). The rule generates a // diagnostic error message when this happens. It makes no changes to the @@ -136,7 +124,6 @@ impl PhysicalOptimizer { // into an `order by max(x) limit y`. In this case it will copy the limit value down // to the aggregation, allowing it to use only y number of accumulators. Arc::new(TopKAggregation::new()), - // Arc::new(SimplifyOrdering::new()), // The ProjectionPushdown rule tries to push projections towards // the sources in the execution plan. As a result of this process, // a projection can disappear if it reaches the source providers, and @@ -144,7 +131,6 @@ impl PhysicalOptimizer { // are not present, the load of executors such as join or union will be // reduced by narrowing their input tables. Arc::new(ProjectionPushdown::new()), - // Appears after AggregateExec is created ]; Self::with_rules(rules) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 412976ca9e43..9f5030998585 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -32,7 +32,6 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{Distribution, LexRequirement, PhysicalSortRequirement}; -use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{ExecutionPlanProperties, PlanProperties}; diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index f70bae883409..adf4b59b74a2 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -15,23 +15,21 @@ // specific language governing permissions and limitations // under the License. -use arrow::compute::kernels::aggregate; use datafusion_common::Result; use datafusion_common::{ config::ConfigOptions, not_impl_err, - tree_node::{Transformed, TransformedResult, TreeNode, TreeNodeRewriter}, + tree_node::{Transformed, TransformedResult, TreeNode}, }; +use datafusion_physical_expr::expressions::{FirstValue, LastValue}; use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::ProjectionMapping, - expressions::{FirstValue, LastValue}, physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalSortRequirement, }; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, - coalesce_partitions::CoalescePartitionsExec, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, }; use std::sync::Arc; @@ -39,29 +37,28 @@ use std::sync::Arc; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; -use super::{output_requirements::OutputRequirementExec, PhysicalOptimizerRule}; +use super::PhysicalOptimizerRule; #[derive(Default)] -pub struct SimplifyOrdering {} +pub struct ConvertFirstLast {} -impl SimplifyOrdering { +impl ConvertFirstLast { pub fn new() -> Self { Self::default() } } -impl PhysicalOptimizerRule for SimplifyOrdering { +impl PhysicalOptimizerRule for ConvertFirstLast { fn optimize( &self, plan: Arc, _config: &ConfigOptions, ) -> Result> { - // Ok(plan) + let res = plan .transform_down(&get_common_requirement_of_aggregate_input) .data(); - // println!("res: {:?}", res); res } @@ -89,45 +86,14 @@ fn get_common_requirement_of_aggregate_input( new_children.push(res.data); } - let mode1 = plan.properties().execution_mode(); - // println!("mode 1: {:?}", mode); - - println!("plan name: {:?}", plan.name()); - println!("is_child_transformed: {:?}", is_child_transformed); - println!("new_children: {:?}", new_children); - let plan = if is_child_transformed { plan.with_new_children(new_children)? } else { plan }; - let mode2 = plan.properties().execution_mode(); - if mode1 != mode2 { - println!("mode1: {:?}, mode2: {:?}", mode1, mode2); - } - let plan = optimize_internal(plan)?; - if plan.transformed { - println!("transformed plan: {:?}", plan.data.name()); - } else { - println!("not transformed plan: {:?}", plan.data.name()); - } - - let mode3 = plan.data.properties().execution_mode(); - if mode1 != mode3 { - println!("mode1: {:?}, mode3: {:?}", mode1, mode3); - } - - if !plan.transformed { - let name = plan.data.name(); - println!( - "not transformed plan: {:?} and is_child_transformed: {:?}", - name, is_child_transformed - ); - } - // If one of the children is transformed, then the plan is considered transformed, then we update // the children of the plan from bottom to top. if plan.transformed || is_child_transformed { @@ -135,36 +101,6 @@ fn get_common_requirement_of_aggregate_input( } else { Ok(Transformed::no(plan.data)) } - - // Ok(plan) - - // if let Some(c) = new_c { - // if !c.transformed { - // return Ok(plan); - // } - - // let plan = plan.data; - - // // TODO: support more types of ExecutionPlan - // if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // let p = aggr_exec.clone_with_input(c.data); - // return Ok(Transformed::yes(Arc::new(p) as Arc)); - // } else if let Some(coalesce_exec) = - // plan.as_any().downcast_ref::() - // { - // let p = coalesce_exec.clone_with_input(c.data); - // return Ok(Transformed::yes(Arc::new(p) as Arc)); - // } else if let Some(out_req_exec) = - // plan.as_any().downcast_ref::() - // { - // let p = out_req_exec.clone_with_input(c.data); - // return Ok(Transformed::yes(Arc::new(p) as Arc)); - // } else { - // return not_impl_err!("Unsupported ExecutionPlan type: {}", plan.name()); - // } - // } - - // return Ok(plan); } fn try_get_updated_aggr_expr_from_child( @@ -232,9 +168,6 @@ fn optimize_internal( let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - println!("required_input_ordering: {:?}", required_input_ordering); - println!("agg_expr: {:?}", aggr_expr); - let input_order_mode = if indices.len() == groupby_exprs.len() && !indices.is_empty() { InputOrderMode::Sorted @@ -402,22 +335,17 @@ fn get_aggregate_exprs_requirement( if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { let mut first_value = first_value.clone(); - // println!("prefix_requirement: {:?}", prefix_requirement); - // println!("aggr_req: {:?}", aggr_req); - // println!("reverse_aggr_req: {:?}", reverse_aggr_req); - + println!("reverse_aggr_req: {:?}", reverse_aggr_req); if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &aggr_req, )) { - println!("1st step"); first_value = first_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(first_value) as _; } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &reverse_aggr_req, )) { - println!("2nd step"); // Converting to LAST_VALUE enables more efficient execution // given the existing ordering: let mut last_value = first_value.convert_to_last(); diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 798dfa2d5bfb..40a1a60d6865 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -465,6 +465,8 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { .create_initial_plan(logical_plan, session_state) .await?; + // println!("first plan: {:#?}", plan); + self.optimize_internal(plan, session_state, |_, _| {}) } } diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 367cc3543577..c14c88d6c69b 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -325,7 +325,6 @@ impl EquivalenceProperties { fn ordering_satisfy_single(&self, req: &PhysicalSortRequirement) -> bool { let expr_ordering = self.get_expr_ordering(req.expr.clone()); let ExprOrdering { expr, data, .. } = expr_ordering; - // println!("expr: {:?}, data: {:?}", expr, data); match data { SortProperties::Ordered(options) => { let sort_expr = PhysicalSortExpr { expr, options }; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index be4ba9fe8a16..6ce68cb27e6b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -48,7 +48,7 @@ use datafusion_physical_expr::{ AggregateExpr, LexRequirement, PhysicalExpr, }; use datafusion_physical_expr::{ - physical_exprs_contains, reverse_order_bys, EquivalenceProperties, LexOrdering, + physical_exprs_contains, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; @@ -964,15 +964,12 @@ fn get_aggregate_exprs_requirement( let mut requirement = vec![]; for aggr_expr in aggr_exprs.iter_mut() { let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_req = reverse_order_bys(aggr_req); + // let reverse_aggr_req = reverse_order_bys(aggr_req); let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + // let reverse_aggr_req = + // PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { - println!("prefix_requirement {:?}", prefix_requirement); - println!("reverse_aggr_req {:?}", reverse_aggr_req); - let mut first_value = first_value.clone(); if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, @@ -1302,11 +1299,11 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, + lit, ApproxDistinct, Count, Median, }; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, - PhysicalSortExpr, PhysicalSortRequirement, + PhysicalSortExpr, }; use futures::{FutureExt, Stream}; diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 2c82f2e6a68d..3ec626aed698 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -239,13 +239,13 @@ logical_plan after optimize_projections SAME TEXT AS ABOVE logical_plan TableScan: simple_explain_test projection=[a, b, c] initial_physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true initial_physical_plan_with_stats CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true, statistics=[Rows=Absent, Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:)]] -physical_plan after SimpleOrdering CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after OutputRequirements OutputRequirementExec --CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE @@ -296,7 +296,6 @@ EXPLAIN VERBOSE SELECT * FROM alltypes_plain limit 10; initial_physical_plan GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after OutputRequirements OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] @@ -304,6 +303,7 @@ OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[ physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE @@ -334,9 +334,6 @@ GlobalLimitExec: skip=0, fetch=10 initial_physical_plan_with_stats GlobalLimitExec: skip=0, fetch=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] --ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[1]:),(Col[2]:),(Col[3]:),(Col[4]:),(Col[5]:),(Col[6]:),(Col[7]:),(Col[8]:),(Col[9]:),(Col[10]:)]] -physical_plan after SimpleOrdering -GlobalLimitExec: skip=0, fetch=10 ---ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/parquet-testing/data/alltypes_plain.parquet]]}, projection=[id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col], limit=10 physical_plan after OutputRequirements OutputRequirementExec --GlobalLimitExec: skip=0, fetch=10 @@ -344,6 +341,7 @@ OutputRequirementExec physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE +physical_plan after SimpleOrdering SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE From a37fb3e3e0db63219a89c6c57926369d3dfab008 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 09:29:43 +0800 Subject: [PATCH 18/77] cleanup Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/coalesce_partitions.rs | 8 -------- 1 file changed, 8 deletions(-) diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 67ceb07aeeb6..1c725ce31f14 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -72,14 +72,6 @@ impl CoalescePartitionsExec { input.execution_mode(), // Execution Mode ) } - - // pub fn clone_with_input(&self, input: Arc) -> Self { - // CoalescePartitionsExec { - // input, - // metrics: self.metrics.clone(), - // cache: self.cache.clone(), - // } - // } } impl DisplayAs for CoalescePartitionsExec { From 8de501c857b9a3b782ea9cc5bb77d18ebd643d18 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 19:50:09 +0800 Subject: [PATCH 19/77] cleanup Signed-off-by: jayzhan211 --- .../physical_optimizer/simplify_ordering.rs | 113 ++---------------- .../physical-plan/src/aggregates/mod.rs | 39 +++--- 2 files changed, 27 insertions(+), 125 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/simplify_ordering.rs index adf4b59b74a2..5f67e0b2edc6 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/simplify_ordering.rs @@ -23,11 +23,10 @@ use datafusion_common::{ }; use datafusion_physical_expr::expressions::{FirstValue, LastValue}; use datafusion_physical_expr::{ - aggregate::is_order_sensitive, - equivalence::ProjectionMapping, - physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, - LexOrdering, LexRequirement, PhysicalSortRequirement, + equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, + EquivalenceProperties, LexRequirement, PhysicalSortRequirement, }; +use datafusion_physical_plan::aggregates::{concat_slices, finer_ordering}; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, @@ -54,7 +53,6 @@ impl PhysicalOptimizerRule for ConvertFirstLast { plan: Arc, _config: &ConfigOptions, ) -> Result> { - let res = plan .transform_down(&get_common_requirement_of_aggregate_input) .data(); @@ -107,7 +105,9 @@ fn try_get_updated_aggr_expr_from_child( aggr_exec: &AggregateExec, ) -> Vec> { let input = aggr_exec.input(); - if aggr_exec.mode() == &AggregateMode::Final || aggr_exec.mode() == &AggregateMode::FinalPartitioned { + if aggr_exec.mode() == &AggregateMode::Final + || aggr_exec.mode() == &AggregateMode::FinalPartitioned + { // Some aggregators may be modified during initialization for // optimization purposes. For example, a FIRST_VALUE may turn // into a LAST_VALUE with the reverse ordering requirement. @@ -115,7 +115,6 @@ fn try_get_updated_aggr_expr_from_child( // `AggregateExpr`/`PhysicalSortExpr` objects. // // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] - if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { if c_aggr_exec.mode() == &AggregateMode::Partial { // If the input is an AggregateExec in Partial mode, then the @@ -135,7 +134,6 @@ fn optimize_internal( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let input = aggr_exec.input(); let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); let group_by = aggr_exec.group_by(); @@ -187,20 +185,6 @@ fn optimize_internal( &input_order_mode, ); - // let p = AggregateExec { - // mode: mode.clone(), - // group_by: group_by.clone(), - // aggr_expr, - // filter_expr: aggr_exec.filter_expr().to_vec(), - // input: input.clone(), - // schema: aggr_exec.schema(), - // input_schema: aggr_exec.input_schema(), - // metrics: aggr_exec.metrics().clone(), - // required_input_ordering, - // limit: None, - // input_order_mode, - // cache, - // }; let p = aggr_exec.clone_with_required_input_ordering_and_aggr_expr( required_input_ordering, aggr_expr, @@ -221,85 +205,6 @@ fn optimize_internal( } } -/// Determines the lexical ordering requirement for an aggregate expression. -/// -/// # Parameters -/// -/// - `aggr_expr`: A reference to an `Arc` representing the -/// aggregate expression. -/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the -/// physical GROUP BY expression. -/// - `agg_mode`: A reference to an `AggregateMode` instance representing the -/// mode of aggregation. -/// -/// # Returns -/// -/// A `LexOrdering` instance indicating the lexical ordering requirement for -/// the aggregate expression. -fn get_aggregate_expr_req( - aggr_expr: &Arc, - group_by: &PhysicalGroupBy, - agg_mode: &AggregateMode, -) -> LexOrdering { - // If the aggregation function is not order sensitive, or the aggregation - // is performing a "second stage" calculation, or all aggregate function - // requirements are inside the GROUP BY expression, then ignore the ordering - // requirement. - if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { - return vec![]; - } - - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); - - // In non-first stage modes, we accumulate data (using `merge_batch`) from - // different partitions (i.e. merge partial results). During this merge, we - // consider the ordering of each partial result. Hence, we do not need to - // use the ordering requirement in such modes as long as partial results are - // generated with the correct ordering. - if group_by.is_single() { - // Remove all orderings that occur in the group by. These requirements - // will definitely be satisfied -- Each group by expression will have - // distinct values per group, hence all requirements are satisfied. - let physical_exprs = group_by.input_exprs(); - req.retain(|sort_expr| { - !physical_exprs_contains(&physical_exprs, &sort_expr.expr) - }); - } - req -} - -/// Computes the finer ordering for between given existing ordering requirement -/// of aggregate expression. -/// -/// # Parameters -/// -/// * `existing_req` - The existing lexical ordering that needs refinement. -/// * `aggr_expr` - A reference to an aggregate expression trait object. -/// * `group_by` - Information about the physical grouping (e.g group by expression). -/// * `eq_properties` - Equivalence properties relevant to the computation. -/// * `agg_mode` - The mode of aggregation (e.g., Partial, Final, etc.). -/// -/// # Returns -/// -/// An `Option` representing the computed finer lexical ordering, -/// or `None` if there is no finer ordering; e.g. the existing requirement and -/// the aggregator requirement is incompatible. -fn finer_ordering( - existing_req: &LexOrdering, - aggr_expr: &Arc, - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Option { - let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - eq_properties.get_finer_ordering(existing_req, &aggr_req) -} - -/// Concatenates the given slices. -fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { - [lhs, rhs].concat() -} - /// Get the common requirement that satisfies all the aggregate expressions. /// /// # Parameters @@ -335,7 +240,6 @@ fn get_aggregate_exprs_requirement( if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { let mut first_value = first_value.clone(); - println!("reverse_aggr_req: {:?}", reverse_aggr_req); if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &aggr_req, @@ -443,13 +347,14 @@ fn get_aggregate_exprs_requirement( mod tests { use super::*; + use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; use datafusion_physical_expr::{ expressions::{col, OrderSensitiveArrayAgg}, PhysicalSortExpr, }; - fn create_test_schema() -> Result { + fn create_test_schema_v4() -> Result { let a = Field::new("a", DataType::Int32, true); let b = Field::new("b", DataType::Int32, true); let c = Field::new("c", DataType::Int32, true); @@ -462,7 +367,7 @@ mod tests { #[tokio::test] async fn test_get_finest_requirements() -> Result<()> { - let test_schema = create_test_schema()?; + let test_schema = create_test_schema_v4()?; // 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 { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6ce68cb27e6b..11c2c7d8a594 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -48,8 +48,7 @@ use datafusion_physical_expr::{ AggregateExpr, LexRequirement, PhysicalExpr, }; use datafusion_physical_expr::{ - physical_exprs_contains, EquivalenceProperties, LexOrdering, - PhysicalSortRequirement, + physical_exprs_contains, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; use itertools::Itertools; @@ -246,19 +245,19 @@ impl From for SendableRecordBatchStream { #[derive(Debug)] pub struct AggregateExec { /// Aggregation mode (full, partial) - pub mode: AggregateMode, + mode: AggregateMode, /// Group by expressions - pub group_by: PhysicalGroupBy, + group_by: PhysicalGroupBy, /// Aggregate expressions - pub aggr_expr: Vec>, + aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression - pub filter_expr: Vec>>, + filter_expr: Vec>>, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause - pub limit: Option, + limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate pub input: Arc, /// Schema after the aggregate is applied - pub schema: SchemaRef, + schema: SchemaRef, /// Input schema before any aggregation is applied. For partial aggregate this will be the /// same as input.schema() but for the final aggregate it will be the same as the input /// to the partial aggregate, i.e., partial and final aggregates have same `input_schema`. @@ -266,11 +265,11 @@ pub struct AggregateExec { /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, /// Execution metrics - pub metrics: ExecutionPlanMetricsSet, - pub required_input_ordering: Option, + metrics: ExecutionPlanMetricsSet, + required_input_ordering: Option, /// Describes how the input is ordered relative to the group by columns - pub input_order_mode: InputOrderMode, - pub cache: PlanProperties, + input_order_mode: InputOrderMode, + cache: PlanProperties, } impl AggregateExec { @@ -282,19 +281,19 @@ impl AggregateExec { input_order_mode: InputOrderMode, ) -> Self { Self { + aggr_expr, required_input_ordering, + metrics: ExecutionPlanMetricsSet::new(), + input_order_mode, + cache, // clone the rest of the fields mode: self.mode, group_by: self.group_by.clone(), - aggr_expr, filter_expr: self.filter_expr.clone(), limit: self.limit, input: self.input.clone(), schema: self.schema.clone(), input_schema: self.input_schema.clone(), - metrics: ExecutionPlanMetricsSet::new(), - input_order_mode, - cache, } } @@ -921,7 +920,7 @@ fn get_aggregate_expr_req( /// An `Option` representing the computed finer lexical ordering, /// or `None` if there is no finer ordering; e.g. the existing requirement and /// the aggregator requirement is incompatible. -fn finer_ordering( +pub fn finer_ordering( existing_req: &LexOrdering, aggr_expr: &Arc, group_by: &PhysicalGroupBy, @@ -933,7 +932,7 @@ fn finer_ordering( } /// Concatenates the given slices. -fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { +pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { [lhs, rhs].concat() } @@ -1298,9 +1297,7 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, Median, - }; + use datafusion_physical_expr::expressions::{lit, ApproxDistinct, Count, Median}; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, From 107f4c82b9e5b86796e59eb7e6b02c2a2ee87035 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 19:52:30 +0800 Subject: [PATCH 20/77] add aggregate test Signed-off-by: jayzhan211 --- .../sqllogictest/test_files/aggregate.slt | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 4929ab485d6d..69fb4ea38e9f 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3428,3 +3428,45 @@ SELECT LAST_VALUE(column1 ORDER BY column2 DESC) IGNORE NULLS FROM t; statement ok DROP TABLE t; + +# Test Convert FirstLast optimizer rule +statement ok +CREATE EXTERNAL TABLE convert_first_last_table ( +c1 INT NOT NULL, +c2 INT NOT NULL, +c3 INT NOT NULL +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (c1 ASC) +WITH ORDER (c2 DESC) +WITH ORDER (c3 ASC) +LOCATION '../core/tests/data/convert_first_last.csv'; + +# test first to last, the result does not show difference, we need to check the conversion by `explain` +query TT +explain select first_value(c1 order by c3 desc) from convert_first_last_table; +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] +--TableScan: convert_first_last_table projection=[c1, c3] +physical_plan +AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true + +# test last to first +query TT +explain select last_value(c1 order by c2 asc) from convert_first_last_table; +---- +logical_plan +Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] +--TableScan: convert_first_last_table projection=[c1, c2] +physical_plan +AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +--CoalescePartitionsExec +----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true From 345d4c60cde8c05aaee218f7a7f3c6eee39c6c82 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 20:06:52 +0800 Subject: [PATCH 21/77] cleanup Signed-off-by: jayzhan211 --- ...simplify_ordering.rs => convert_first_last.rs} | 15 +++++---------- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../core/src/physical_optimizer/optimizer.rs | 2 +- datafusion/physical-plan/src/aggregates/mod.rs | 4 +++- datafusion/physical-plan/src/recursive_query.rs | 1 - datafusion/physical-plan/src/tree_node.rs | 2 -- 6 files changed, 10 insertions(+), 16 deletions(-) rename datafusion/core/src/physical_optimizer/{simplify_ordering.rs => convert_first_last.rs} (96%) diff --git a/datafusion/core/src/physical_optimizer/simplify_ordering.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs similarity index 96% rename from datafusion/core/src/physical_optimizer/simplify_ordering.rs rename to datafusion/core/src/physical_optimizer/convert_first_last.rs index 5f67e0b2edc6..a7bb01550a4b 100644 --- a/datafusion/core/src/physical_optimizer/simplify_ordering.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -185,20 +185,14 @@ fn optimize_internal( &input_order_mode, ); - let p = aggr_exec.clone_with_required_input_ordering_and_aggr_expr( + let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info( required_input_ordering, aggr_expr, cache, input_order_mode, ); - // let aggr_exec = aggr_exec.to_owned().to_owned(); - // aggr_exec.with_aggr_expr(aggr_expr); - // aggr_exec.with_required_input_ordering(required_input_ordering); - // aggr_exec.with_cache(cache); - // aggr_exec.with_input_order_mode(input_order_mode); - - let res = Arc::new(p) as Arc; + let res = Arc::new(aggr_exec) as Arc; Ok(Transformed::yes(res)) } else { Ok(Transformed::no(plan)) @@ -345,6 +339,7 @@ fn get_aggregate_exprs_requirement( Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } +#[cfg(test)] mod tests { use super::*; @@ -354,7 +349,7 @@ mod tests { PhysicalSortExpr, }; - fn create_test_schema_v4() -> Result { + fn create_test_schema() -> Result { let a = Field::new("a", DataType::Int32, true); let b = Field::new("b", DataType::Int32, true); let c = Field::new("c", DataType::Int32, true); @@ -367,7 +362,7 @@ mod tests { #[tokio::test] async fn test_get_finest_requirements() -> Result<()> { - let test_schema = create_test_schema_v4()?; + 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 { diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 06f897471963..c552523d1c1d 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -34,7 +34,7 @@ pub mod pipeline_checker; mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; -mod simplify_ordering; +mod convert_first_last; mod sort_pushdown; pub mod topk_aggregation; mod utils; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 134d11e43a0e..897a315e64b7 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use super::projection_pushdown::ProjectionPushdown; -use super::simplify_ordering::ConvertFirstLast; +use super::convert_first_last::ConvertFirstLast; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 11c2c7d8a594..a99fe45fc856 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -273,7 +273,9 @@ pub struct AggregateExec { } impl AggregateExec { - pub fn clone_with_required_input_ordering_and_aggr_expr( + /// Function used in `ConvertFirstLast` optimizer rule, + /// where we need parts of the new value, others cloned from the old one + pub fn new_with_aggr_expr_and_ordering_info( &self, required_input_ordering: Option, aggr_expr: Vec>, diff --git a/datafusion/physical-plan/src/recursive_query.rs b/datafusion/physical-plan/src/recursive_query.rs index 91abd5cfdf28..ba7d1a54548a 100644 --- a/datafusion/physical-plan/src/recursive_query.rs +++ b/datafusion/physical-plan/src/recursive_query.rs @@ -26,7 +26,6 @@ use super::{ work_table::{ReservedBatches, WorkTable, WorkTableExec}, PlanProperties, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use crate::aggregates::AggregateExec; use crate::{DisplayAs, DisplayFormatType, ExecutionMode, ExecutionPlan}; use arrow::datatypes::SchemaRef; diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs index 86cf2d055474..746df9dd8755 100644 --- a/datafusion/physical-plan/src/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -17,11 +17,9 @@ //! This module provides common traits for visiting or rewriting tree nodes easily. -use core::panic; use std::fmt::{self, Display, Formatter}; use std::sync::Arc; -use crate::aggregates::AggregateExec; use crate::{displayable, with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{ConcreteTreeNode, DynTreeNode}; From 030d7b9cbb4dc8bbcb8b0ad684de4c46f7b6391d Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 20:24:29 +0800 Subject: [PATCH 22/77] final draft Signed-off-by: jayzhan211 --- .../physical_optimizer/convert_first_last.rs | 32 ++++++++----- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../core/src/physical_optimizer/optimizer.rs | 2 +- .../physical_optimizer/output_requirements.rs | 1 - datafusion/core/src/physical_planner.rs | 5 +- .../physical-plan/src/aggregates/mod.rs | 47 +------------------ datafusion/physical-plan/src/lib.rs | 1 - datafusion/sqllogictest/test_files/test1.slt | 47 ------------------- 8 files changed, 25 insertions(+), 112 deletions(-) delete mode 100644 datafusion/sqllogictest/test_files/test1.slt diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index a7bb01550a4b..d3b97cda844b 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -38,6 +38,15 @@ use datafusion_physical_plan::windows::get_ordered_partition_by_indices; use super::PhysicalOptimizerRule; +/// The optimizer rule check the ordering requirements of the aggregate expressions. +/// And convert between FIRST_VALUE and LAST_VALUE if possible. +/// For example, If we have an ascending values and we want LastValue from the descending requirement, +/// it is equivalent to FirstValue with the current ascending ordering. +/// +/// The concrete example is that, says we have values c1 with [1, 2, 3], which is an ascending order. +/// If we want LastValue(c1 order by desc), which is the first value of reversed c1 [3, 2, 1], +/// so we can convert the aggregate expression to FirstValue(c1 order by asc), +/// since the current ordering is already satisfied, it saves our time! #[derive(Default)] pub struct ConvertFirstLast {} @@ -53,11 +62,8 @@ impl PhysicalOptimizerRule for ConvertFirstLast { plan: Arc, _config: &ConfigOptions, ) -> Result> { - let res = plan - .transform_down(&get_common_requirement_of_aggregate_input) - .data(); - - res + plan.transform_down(&get_common_requirement_of_aggregate_input) + .data() } fn name(&self) -> &str { @@ -156,7 +162,7 @@ fn optimize_internal( let req = get_aggregate_exprs_requirement( &new_requirement, &mut aggr_expr, - &group_by, + group_by, input_eq_properties, mode, )?; @@ -175,13 +181,13 @@ fn optimize_internal( InputOrderMode::Linear }; let projection_mapping = - ProjectionMapping::try_new(&group_by.expr(), &input.schema())?; + ProjectionMapping::try_new(group_by.expr(), &input.schema())?; let cache = AggregateExec::compute_properties( - &input, + input, plan.schema().clone(), &projection_mapping, - &mode, + mode, &input_order_mode, ); @@ -192,8 +198,9 @@ fn optimize_internal( input_order_mode, ); - let res = Arc::new(aggr_exec) as Arc; - Ok(Transformed::yes(res)) + Ok(Transformed::yes( + Arc::new(aggr_exec) as Arc + )) } else { Ok(Transformed::no(plan)) } @@ -216,6 +223,9 @@ fn optimize_internal( /// /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. +/// +/// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this +/// function care about the possible of optimization of FIRST_VALUE and LAST_VALUE fn get_aggregate_exprs_requirement( prefix_requirement: &[PhysicalSortRequirement], aggr_exprs: &mut [Arc], diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index c552523d1c1d..c80668c6da74 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -24,6 +24,7 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; +mod convert_first_last; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; @@ -34,7 +35,6 @@ pub mod pipeline_checker; mod projection_pushdown; pub mod pruning; pub mod replace_with_order_preserving_variants; -mod convert_first_last; mod sort_pushdown; pub mod topk_aggregation; mod utils; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 897a315e64b7..e8ae03e11c62 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,8 +19,8 @@ use std::sync::Arc; -use super::projection_pushdown::ProjectionPushdown; use super::convert_first_last::ConvertFirstLast; +use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index 9f5030998585..829d523c990c 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -274,7 +274,6 @@ fn require_top_ordering_helper( // be responsible for (i.e. the originator of) the global ordering. let (new_child, is_changed) = require_top_ordering_helper(children.swap_remove(0))?; - Ok((plan.with_new_children(vec![new_child])?, is_changed)) } else { // Stop searching, there is no global ordering desired for the query. diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 40a1a60d6865..c25523c5ae33 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -74,7 +74,6 @@ use arrow::compute::SortOptions; use arrow::datatypes::{Schema, SchemaRef}; use arrow_array::builder::StringBuilder; use arrow_array::RecordBatch; -use async_trait::async_trait; use datafusion_common::display::ToStringifiedPlan; use datafusion_common::{ exec_err, internal_err, not_impl_err, plan_err, DFSchema, FileType, ScalarValue, @@ -95,6 +94,7 @@ use datafusion_physical_expr::expressions::Literal; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; use datafusion_sql::utils::window_expr_common_partition_keys; +use async_trait::async_trait; use datafusion_common::config::FormatOptions; use datafusion_physical_expr::LexOrdering; use futures::future::BoxFuture; @@ -464,9 +464,6 @@ impl PhysicalPlanner for DefaultPhysicalPlanner { let plan = self .create_initial_plan(logical_plan, session_state) .await?; - - // println!("first plan: {:#?}", plan); - self.optimize_internal(plan, session_state, |_, _| {}) } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a99fe45fc856..66488d1d10e3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -273,7 +273,7 @@ pub struct AggregateExec { } impl AggregateExec { - /// Function used in `ConvertFirstLast` optimizer rule, + /// Function used in `ConvertFirstLast` optimizer rule, /// where we need parts of the new value, others cloned from the old one pub fn new_with_aggr_expr_and_ordering_info( &self, @@ -421,29 +421,6 @@ impl AggregateExec { }) } - pub fn with_cache(mut self, cache: PlanProperties) -> Self { - self.cache = cache; - self - } - - pub fn with_input_order_mode(mut self, input_order_mode: InputOrderMode) -> Self { - self.input_order_mode = input_order_mode; - self - } - - pub fn with_required_input_ordering( - mut self, - required_input_ordering: Option, - ) -> Self { - self.required_input_ordering = required_input_ordering; - self - } - - pub fn with_aggr_expr(mut self, aggr_expr: Vec>) -> Self { - self.aggr_expr = aggr_expr; - self - } - /// Aggregation mode (full, partial) pub fn mode(&self) -> &AggregateMode { &self.mode @@ -965,10 +942,7 @@ fn get_aggregate_exprs_requirement( let mut requirement = vec![]; for aggr_expr in aggr_exprs.iter_mut() { let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - // let reverse_aggr_req = reverse_order_bys(aggr_req); let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - // let reverse_aggr_req = - // PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { let mut first_value = first_value.clone(); @@ -978,15 +952,6 @@ fn get_aggregate_exprs_requirement( )) { first_value = first_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(first_value) as _; - // } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - // prefix_requirement, - // &reverse_aggr_req, - // )) { - // // Converting to LAST_VALUE enables more efficient execution - // // given the existing ordering: - // let mut last_value = first_value.convert_to_last(); - // last_value = last_value.with_requirement_satisfied(true); - // *aggr_expr = Arc::new(last_value) as _; } else { // Requirement is not satisfied with existing ordering. first_value = first_value.with_requirement_satisfied(false); @@ -1002,16 +967,6 @@ fn get_aggregate_exprs_requirement( )) { last_value = last_value.with_requirement_satisfied(true); *aggr_expr = Arc::new(last_value) as _; - // } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - // prefix_requirement, - // &reverse_aggr_req, - // )) { - // println!("last to first {:?}", aggr_expr); - // // Converting to FIRST_VALUE enables more efficient execution - // // given the existing ordering: - // let mut first_value = last_value.convert_to_first(); - // first_value = first_value.with_requirement_satisfied(true); - // *aggr_expr = Arc::new(first_value) as _; } else { // Requirement is not satisfied with existing ordering. last_value = last_value.with_requirement_satisfied(false); diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 8890777b40eb..e1c8489655bf 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -27,7 +27,6 @@ use crate::metrics::MetricsSet; use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; -use aggregates::AggregateExec; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::config::ConfigOptions; diff --git a/datafusion/sqllogictest/test_files/test1.slt b/datafusion/sqllogictest/test_files/test1.slt deleted file mode 100644 index 4ca847fa01d8..000000000000 --- a/datafusion/sqllogictest/test_files/test1.slt +++ /dev/null @@ -1,47 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at - -# http://www.apache.org/licenses/LICENSE-2.0 - -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. - -statement ok -CREATE EXTERNAL TABLE convert_first_last_table ( -c1 INT NOT NULL, -c2 INT NOT NULL, -c3 INT NOT NULL -) -STORED AS CSV -WITH HEADER ROW -WITH ORDER (c1 ASC) -WITH ORDER (c2 DESC) -WITH ORDER (c3 ASC) -LOCATION '../core/tests/data/convert_first_last.csv'; - -# test first to last, the result does not show difference, we need to check the conversion by `explain` -query TT -explain select first_value(c1 order by c3 desc) from convert_first_last_table; ----- -logical_plan -Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] ---TableScan: convert_first_last_table projection=[c1, c3] -physical_plan -AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true - -# test last to first -# query TT -# explain select last_value(c1 order by c2 asc) from convert_first_last_table; From cf3f33e0e67cc5edf63df40b316f2ef9cba72c61 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Fri, 12 Apr 2024 20:27:25 +0800 Subject: [PATCH 23/77] cleanup again Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/aggregates/mod.rs | 6 ------ datafusion/physical-plan/src/tree_node.rs | 3 +-- datafusion/sqllogictest/test_files/group_by.slt | 4 ---- 3 files changed, 1 insertion(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 66488d1d10e3..9a9a29971de9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -404,7 +404,6 @@ impl AggregateExec { &input_order_mode, ); - // let required_input_ordering = None; Ok(AggregateExec { mode, group_by, @@ -711,9 +710,6 @@ impl ExecutionPlan for AggregateExec { self: Arc, children: Vec>, ) -> Result> { - // let aggr_exec = self.clone_with_input(children[0].clone()); - // Ok(Arc::new(aggr_exec)) - let mut me = AggregateExec::try_new_with_schema( self.mode, self.group_by.clone(), @@ -725,8 +721,6 @@ impl ExecutionPlan for AggregateExec { )?; me.limit = self.limit; - // let me = me.rewrite_ordering()?; - Ok(Arc::new(me)) } diff --git a/datafusion/physical-plan/src/tree_node.rs b/datafusion/physical-plan/src/tree_node.rs index 746df9dd8755..46460cbb6684 100644 --- a/datafusion/physical-plan/src/tree_node.rs +++ b/datafusion/physical-plan/src/tree_node.rs @@ -63,8 +63,7 @@ impl PlanContext { pub fn update_plan_from_children(mut self) -> Result { let children_plans = self.children.iter().map(|c| c.plan.clone()).collect(); - let plan = with_new_children_if_necessary(self.plan, children_plans)?; - self.plan = plan; + self.plan = with_new_children_if_necessary(self.plan, children_plans)?; Ok(self) } diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 88e0272ef9bc..869462b4722a 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -4249,10 +4249,6 @@ SELECT date_bin('15 minutes', ts) as time_chunks 2018-12-13T12:00:00 2018-11-13T17:00:00 - - - - # Since extract is not a monotonic function, below query should not run. # when source is unbounded. query error From 6d9ee9f9ee4ebedfeb0afb688a87723a2b1cf3ef Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 09:10:07 +0800 Subject: [PATCH 24/77] pull out finer ordering code and reuse Signed-off-by: jayzhan211 --- .../physical_optimizer/convert_first_last.rs | 59 +------ .../physical-plan/src/aggregates/mod.rs | 156 +++++++++--------- 2 files changed, 79 insertions(+), 136 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index d3b97cda844b..97b95a94dca6 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -26,7 +26,7 @@ use datafusion_physical_expr::{ equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, EquivalenceProperties, LexRequirement, PhysicalSortRequirement, }; -use datafusion_physical_plan::aggregates::{concat_slices, finer_ordering}; +use datafusion_physical_plan::aggregates::{concat_slices, finer_ordering, optimize_for_finer_ordering}; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, @@ -290,61 +290,8 @@ fn get_aggregate_exprs_requirement( } continue; } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Requirement is satisfied by existing ordering - requirement = finer_ordering; - continue; - } - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Reverse requirement is satisfied by exiting ordering. - // Hence reverse the aggregator - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } - } - } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - // There is a requirement that both satisfies existing requirement and current - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - continue; - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - // There is a requirement that both satisfies existing requirement and reverse - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } - } - // Neither the existing requirement and current aggregate requirement satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); + + optimize_for_finer_ordering(&mut requirement, aggr_expr, group_by, eq_properties, agg_mode)?; } Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 9a9a29971de9..9d848ae9ed1f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -938,93 +938,89 @@ fn get_aggregate_exprs_requirement( let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { - let mut first_value = first_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; - } - continue; - } - if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { - let mut last_value = last_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - last_value = last_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(last_value) as _; - } - continue; + optimize_for_finer_ordering( + &mut requirement, + aggr_expr, + group_by, + eq_properties, + agg_mode, + )?; + } + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) +} + +/// Optimize the requirement for finer ordering based on various parameters. +/// +/// This function takes in a mutable reference to a `LexOrdering`, a mutable reference to an `AggregateExpr`, +/// a `PhysicalGroupBy`, an `EquivalenceProperties`, and an `AggregateMode`, and attempts to optimize the +/// requirement for finer ordering. It checks if the finer ordering satisfies existing requirements or +/// if a reverse ordering satisfies them, updating the references accordingly. If neither satisfies the +/// requirements, it returns an error indicating conflicting ordering requirements. +pub fn optimize_for_finer_ordering( + requirement: &mut LexOrdering, + aggr_expr: &mut Arc, + group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, +) -> Result<()> { + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Requirement is satisfied by existing ordering + *requirement = finer_ordering; + return Ok(()); } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { if eq_properties.ordering_satisfy(&finer_ordering) { - // Requirement is satisfied by existing ordering - requirement = finer_ordering; - continue; - } - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Reverse requirement is satisfied by exiting ordering. - // Hence reverse the aggregator - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } + // Reverse requirement is satisfied by exiting ordering. + // Hence reverse the aggregator + *requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + return Ok(()); } } - if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - // There is a requirement that both satisfies existing requirement and current + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + // There is a requirement that both satisfies existing requirement and current + // aggregate requirement. Use updated requirement + // return (Some(finer_ordering), None); + *requirement = finer_ordering; + return Ok(()); + } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + // There is a requirement that both satisfies existing requirement and reverse // aggregate requirement. Use updated requirement - requirement = finer_ordering; - continue; - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - // There is a requirement that both satisfies existing requirement and reverse - // aggregate requirement. Use updated requirement - requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - continue; - } + // return (Some(finer_ordering), Some(reverse_aggr_expr)); + *requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + return Ok(()); } - // Neither the existing requirement and current aggregate requirement satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - return not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ); } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) + + // Neither the existing requirement and current aggregate requirement satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ); } /// returns physical expressions for arguments to evaluate against a batch From abd40c607ee50184c66952b0bb364eb20dcab170 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 09:22:31 +0800 Subject: [PATCH 25/77] clippy Signed-off-by: jayzhan211 --- .../physical_optimizer/convert_first_last.rs | 11 ++++++--- .../physical-plan/src/aggregates/mod.rs | 24 ++++++++----------- 2 files changed, 18 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 97b95a94dca6..28aebe049e44 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -18,7 +18,6 @@ use datafusion_common::Result; use datafusion_common::{ config::ConfigOptions, - not_impl_err, tree_node::{Transformed, TransformedResult, TreeNode}, }; use datafusion_physical_expr::expressions::{FirstValue, LastValue}; @@ -26,7 +25,7 @@ use datafusion_physical_expr::{ equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, EquivalenceProperties, LexRequirement, PhysicalSortRequirement, }; -use datafusion_physical_plan::aggregates::{concat_slices, finer_ordering, optimize_for_finer_ordering}; +use datafusion_physical_plan::aggregates::{concat_slices, optimize_for_finer_ordering}; use datafusion_physical_plan::{ aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, @@ -291,7 +290,13 @@ fn get_aggregate_exprs_requirement( continue; } - optimize_for_finer_ordering(&mut requirement, aggr_expr, group_by, eq_properties, agg_mode)?; + optimize_for_finer_ordering( + &mut requirement, + aggr_expr, + group_by, + eq_properties, + agg_mode, + )?; } Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 9d848ae9ed1f..6fd8a887bad2 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -41,7 +41,6 @@ use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::aggregate::is_order_sensitive; use datafusion_physical_expr::equivalence::collapse_lex_req; -use datafusion_physical_expr::expressions::{FirstValue, LastValue}; use datafusion_physical_expr::{ equivalence::ProjectionMapping, expressions::{Column, Max, Min, UnKnownColumn}, @@ -371,7 +370,6 @@ impl AggregateExec { .collect::>(); let req = get_aggregate_exprs_requirement( - &new_requirement, &mut aggr_expr, &group_by, input_eq_properties, @@ -927,7 +925,6 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. fn get_aggregate_exprs_requirement( - prefix_requirement: &[PhysicalSortRequirement], aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, @@ -935,9 +932,6 @@ fn get_aggregate_exprs_requirement( ) -> Result { let mut requirement = vec![]; for aggr_expr in aggr_exprs.iter_mut() { - let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - optimize_for_finer_ordering( &mut requirement, aggr_expr, @@ -955,7 +949,7 @@ fn get_aggregate_exprs_requirement( /// a `PhysicalGroupBy`, an `EquivalenceProperties`, and an `AggregateMode`, and attempts to optimize the /// requirement for finer ordering. It checks if the finer ordering satisfies existing requirements or /// if a reverse ordering satisfies them, updating the references accordingly. If neither satisfies the -/// requirements, it returns an error indicating conflicting ordering requirements. +/// requirements, it returns an error indicating conflicting ordering requirements. pub fn optimize_for_finer_ordering( requirement: &mut LexOrdering, aggr_expr: &mut Arc, @@ -964,7 +958,7 @@ pub fn optimize_for_finer_ordering( agg_mode: &AggregateMode, ) -> Result<()> { if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + finer_ordering(requirement, aggr_expr, group_by, eq_properties, agg_mode) { if eq_properties.ordering_satisfy(&finer_ordering) { // Requirement is satisfied by existing ordering @@ -974,7 +968,7 @@ pub fn optimize_for_finer_ordering( } if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { if let Some(finer_ordering) = finer_ordering( - &requirement, + requirement, &reverse_aggr_expr, group_by, eq_properties, @@ -990,7 +984,7 @@ pub fn optimize_for_finer_ordering( } } if let Some(finer_ordering) = - finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + finer_ordering(requirement, aggr_expr, group_by, eq_properties, agg_mode) { // There is a requirement that both satisfies existing requirement and current // aggregate requirement. Use updated requirement @@ -1000,7 +994,7 @@ pub fn optimize_for_finer_ordering( } if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { if let Some(finer_ordering) = finer_ordering( - &requirement, + requirement, &reverse_aggr_expr, group_by, eq_properties, @@ -1018,9 +1012,9 @@ pub fn optimize_for_finer_ordering( // Neither the existing requirement and current aggregate requirement satisfy the other, this means // requirements are conflicting. Currently, we do not support // conflicting requirements. - return not_impl_err!( + not_impl_err!( "Conflicting ordering requirements in aggregate functions is not supported" - ); + ) } /// returns physical expressions for arguments to evaluate against a batch @@ -1244,7 +1238,9 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use datafusion_physical_expr::expressions::{lit, ApproxDistinct, Count, Median}; + use datafusion_physical_expr::expressions::{ + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, + }; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, From 3e2c18606236f47572a1534c1e325c8c84884c6e Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 15:16:01 +0800 Subject: [PATCH 26/77] remove finer in optimize rule Signed-off-by: jayzhan211 --- .../physical_optimizer/convert_first_last.rs | 153 +++--------------- .../physical-plan/src/aggregates/mod.rs | 101 +++++++++++- 2 files changed, 116 insertions(+), 138 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 28aebe049e44..f95fe2eb27b0 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -23,16 +23,15 @@ use datafusion_common::{ use datafusion_physical_expr::expressions::{FirstValue, LastValue}; use datafusion_physical_expr::{ equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, - EquivalenceProperties, LexRequirement, PhysicalSortRequirement, + EquivalenceProperties, PhysicalSortRequirement, }; -use datafusion_physical_plan::aggregates::{concat_slices, optimize_for_finer_ordering}; +use datafusion_physical_plan::aggregates::concat_slices; use datafusion_physical_plan::{ - aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, + aggregates::{AggregateExec, AggregateMode}, ExecutionPlan, ExecutionPlanProperties, InputOrderMode, }; use std::sync::Arc; -use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_plan::windows::get_ordered_partition_by_indices; use super::PhysicalOptimizerRule; @@ -61,7 +60,7 @@ impl PhysicalOptimizerRule for ConvertFirstLast { plan: Arc, _config: &ConfigOptions, ) -> Result> { - plan.transform_down(&get_common_requirement_of_aggregate_input) + plan.transform_up(&get_common_requirement_of_aggregate_input) .data() } @@ -77,24 +76,26 @@ impl PhysicalOptimizerRule for ConvertFirstLast { fn get_common_requirement_of_aggregate_input( plan: Arc, ) -> Result>> { + // Optimize children let children = plan.children(); - let mut is_child_transformed = false; let mut new_children: Vec> = vec![]; for c in children.iter() { - let res = get_common_requirement_of_aggregate_input(c.clone())?; + let res = optimize_internal(c.clone())?; if res.transformed { is_child_transformed = true; } new_children.push(res.data); } + // Update children if transformed let plan = if is_child_transformed { plan.with_new_children(new_children)? } else { plan }; + // Update itself let plan = optimize_internal(plan)?; // If one of the children is transformed, then the plan is considered transformed, then we update @@ -106,6 +107,9 @@ fn get_common_requirement_of_aggregate_input( } } +/// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer +/// is in Partial mode and the second layer is in Final or Finalpartitioned mode. +/// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. fn try_get_updated_aggr_expr_from_child( aggr_exec: &AggregateExec, ) -> Vec> { @@ -150,7 +154,7 @@ fn optimize_internal( // prefix requirements with this section. In this case, aggregation will // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, input); - let mut new_requirement = indices + let requirement = indices .iter() .map(|&idx| PhysicalSortRequirement { expr: groupby_exprs[idx].clone(), @@ -158,18 +162,13 @@ fn optimize_internal( }) .collect::>(); - let req = get_aggregate_exprs_requirement( - &new_requirement, + get_aggregate_exprs_requirement( + &requirement, &mut aggr_expr, - group_by, input_eq_properties, - mode, )?; - new_requirement.extend(req); - new_requirement = collapse_lex_req(new_requirement); - let required_input_ordering = - (!new_requirement.is_empty()).then_some(new_requirement); + let required_input_ordering = (!requirement.is_empty()).then_some(requirement); let input_order_mode = if indices.len() == groupby_exprs.len() && !indices.is_empty() { @@ -224,15 +223,12 @@ fn optimize_internal( /// aggregate requirements. Returns an error in case of conflicting requirements. /// /// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this -/// function care about the possible of optimization of FIRST_VALUE and LAST_VALUE +/// function care only the possible conversion between FIRST_VALUE and LAST_VALUE fn get_aggregate_exprs_requirement( prefix_requirement: &[PhysicalSortRequirement], aggr_exprs: &mut [Arc], - group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Result { - let mut requirement = vec![]; +) -> Result<()> { for aggr_expr in aggr_exprs.iter_mut() { let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); let reverse_aggr_req = reverse_order_bys(aggr_req); @@ -289,120 +285,7 @@ fn get_aggregate_exprs_requirement( } continue; } - - optimize_for_finer_ordering( - &mut requirement, - aggr_expr, - group_by, - eq_properties, - agg_mode, - )?; - } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) -} - -#[cfg(test)] -mod tests { - use super::*; - - use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; - use datafusion_physical_expr::{ - expressions::{col, OrderSensitiveArrayAgg}, - PhysicalSortExpr, - }; - - fn create_test_schema() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); - - Ok(schema) } - #[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 col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let mut eq_properties = EquivalenceProperties::new(test_schema); - // Columns a and b are equal. - eq_properties.add_equal_conditions(col_a, col_b); - // Aggregate requirements are - // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively - let order_by_exprs = vec![ - None, - Some(vec![PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }]), - Some(vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_b.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_c.clone(), - options: options1, - }, - ]), - Some(vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_b.clone(), - options: options1, - }, - ]), - ]; - let common_requirement = vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options: options1, - }, - PhysicalSortExpr { - expr: col_c.clone(), - options: options1, - }, - ]; - let mut aggr_exprs = order_by_exprs - .into_iter() - .map(|order_by_expr| { - Arc::new(OrderSensitiveArrayAgg::new( - col_a.clone(), - "array_agg", - DataType::Int32, - false, - vec![], - order_by_expr.unwrap_or_default(), - )) as _ - }) - .collect::>(); - let group_by = PhysicalGroupBy::new_single(vec![]); - let res = get_aggregate_exprs_requirement( - &[], - &mut aggr_exprs, - &group_by, - &eq_properties, - &AggregateMode::Partial, - )?; - let res = PhysicalSortRequirement::to_sort_exprs(res); - assert_eq!(res, common_requirement); - Ok(()) - } + Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6fd8a887bad2..3f484b8580f3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -891,7 +891,7 @@ fn get_aggregate_expr_req( /// An `Option` representing the computed finer lexical ordering, /// or `None` if there is no finer ordering; e.g. the existing requirement and /// the aggregator requirement is incompatible. -pub fn finer_ordering( +fn finer_ordering( existing_req: &LexOrdering, aggr_expr: &Arc, group_by: &PhysicalGroupBy, @@ -950,7 +950,7 @@ fn get_aggregate_exprs_requirement( /// requirement for finer ordering. It checks if the finer ordering satisfies existing requirements or /// if a reverse ordering satisfies them, updating the references accordingly. If neither satisfies the /// requirements, it returns an error indicating conflicting ordering requirements. -pub fn optimize_for_finer_ordering( +fn optimize_for_finer_ordering( requirement: &mut LexOrdering, aggr_expr: &mut Arc, group_by: &PhysicalGroupBy, @@ -1239,7 +1239,7 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Count, FirstValue, LastValue, Median, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, }; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalExpr, @@ -1248,6 +1248,18 @@ mod tests { use futures::{FutureExt, Stream}; + // Generate a schema which consists of 5 columns (a, b, c, d, e) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); + + Ok(schema) + } + /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. @@ -2091,6 +2103,89 @@ 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 col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let mut eq_properties = EquivalenceProperties::new(test_schema); + // Columns a and b are equal. + eq_properties.add_equal_conditions(col_a, col_b); + // Aggregate requirements are + // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively + let order_by_exprs = vec![ + None, + Some(vec![PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options: options1, + }, + ]), + ]; + let common_requirement = vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]; + let mut aggr_exprs = order_by_exprs + .into_iter() + .map(|order_by_expr| { + Arc::new(OrderSensitiveArrayAgg::new( + col_a.clone(), + "array_agg", + DataType::Int32, + false, + vec![], + order_by_expr.unwrap_or_default(), + )) as _ + }) + .collect::>(); + let group_by = PhysicalGroupBy::new_single(vec![]); + let res = get_aggregate_exprs_requirement( + &mut aggr_exprs, + &group_by, + &eq_properties, + &AggregateMode::Partial, + )?; + let res = PhysicalSortRequirement::to_sort_exprs(res); + assert_eq!(res, common_requirement); + Ok(()) + } + #[test] fn test_agg_exec_same_schema() -> Result<()> { let schema = Arc::new(Schema::new(vec![ From d7c259035ec8ff0e1edc0366db2e0d59a27d9414 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 15:28:15 +0800 Subject: [PATCH 27/77] add comments and clenaup Signed-off-by: jayzhan211 --- datafusion/core/src/physical_optimizer/convert_first_last.rs | 3 +++ datafusion/physical-plan/src/aggregates/mod.rs | 2 -- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index f95fe2eb27b0..c067bd2f6452 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -110,6 +110,9 @@ fn get_common_requirement_of_aggregate_input( /// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer /// is in Partial mode and the second layer is in Final or Finalpartitioned mode. /// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. +/// Therefore, we check it and get the updated aggregate expressions. +/// +/// If AggregateExec is created from elsewhere, we skip the check and return the original aggregate expressions. fn try_get_updated_aggr_expr_from_child( aggr_exec: &AggregateExec, ) -> Vec> { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3f484b8580f3..d457119b5b9b 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -988,7 +988,6 @@ fn optimize_for_finer_ordering( { // There is a requirement that both satisfies existing requirement and current // aggregate requirement. Use updated requirement - // return (Some(finer_ordering), None); *requirement = finer_ordering; return Ok(()); } @@ -1002,7 +1001,6 @@ fn optimize_for_finer_ordering( ) { // There is a requirement that both satisfies existing requirement and reverse // aggregate requirement. Use updated requirement - // return (Some(finer_ordering), Some(reverse_aggr_expr)); *requirement = finer_ordering; *aggr_expr = reverse_aggr_expr; return Ok(()); From fed356a35857a6b17258efee13dab7601f502916 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 15:34:06 +0800 Subject: [PATCH 28/77] rename fun Signed-off-by: jayzhan211 --- datafusion/core/src/physical_optimizer/convert_first_last.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index c067bd2f6452..45e95827c773 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -165,7 +165,7 @@ fn optimize_internal( }) .collect::>(); - get_aggregate_exprs_requirement( + try_convert_first_last_if_better( &requirement, &mut aggr_expr, input_eq_properties, @@ -227,7 +227,7 @@ fn optimize_internal( /// /// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this /// function care only the possible conversion between FIRST_VALUE and LAST_VALUE -fn get_aggregate_exprs_requirement( +fn try_convert_first_last_if_better( prefix_requirement: &[PhysicalSortRequirement], aggr_exprs: &mut [Arc], eq_properties: &EquivalenceProperties, From 0d3d4618a90b99fc642094021ea76ecd5f6224a2 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 15:39:54 +0800 Subject: [PATCH 29/77] rename fun Signed-off-by: jayzhan211 --- .../physical-plan/src/aggregates/mod.rs | 132 ++++++++---------- 1 file changed, 55 insertions(+), 77 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index d457119b5b9b..e24e71e73540 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -369,7 +369,7 @@ impl AggregateExec { }) .collect::>(); - let req = get_aggregate_exprs_requirement( + let req = get_finer_aggregate_exprs_requirement( &mut aggr_expr, &group_by, input_eq_properties, @@ -924,7 +924,7 @@ pub fn concat_slices(lhs: &[T], rhs: &[T]) -> Vec { /// /// A `LexRequirement` instance, which is the requirement that satisfies all the /// aggregate requirements. Returns an error in case of conflicting requirements. -fn get_aggregate_exprs_requirement( +fn get_finer_aggregate_exprs_requirement( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, @@ -932,87 +932,65 @@ fn get_aggregate_exprs_requirement( ) -> Result { let mut requirement = vec![]; for aggr_expr in aggr_exprs.iter_mut() { - optimize_for_finer_ordering( - &mut requirement, - aggr_expr, - group_by, - eq_properties, - agg_mode, - )?; - } - Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) -} - -/// Optimize the requirement for finer ordering based on various parameters. -/// -/// This function takes in a mutable reference to a `LexOrdering`, a mutable reference to an `AggregateExpr`, -/// a `PhysicalGroupBy`, an `EquivalenceProperties`, and an `AggregateMode`, and attempts to optimize the -/// requirement for finer ordering. It checks if the finer ordering satisfies existing requirements or -/// if a reverse ordering satisfies them, updating the references accordingly. If neither satisfies the -/// requirements, it returns an error indicating conflicting ordering requirements. -fn optimize_for_finer_ordering( - requirement: &mut LexOrdering, - aggr_expr: &mut Arc, - group_by: &PhysicalGroupBy, - eq_properties: &EquivalenceProperties, - agg_mode: &AggregateMode, -) -> Result<()> { - if let Some(finer_ordering) = - finer_ordering(requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - if eq_properties.ordering_satisfy(&finer_ordering) { - // Requirement is satisfied by existing ordering - *requirement = finer_ordering; - return Ok(()); - } - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { if eq_properties.ordering_satisfy(&finer_ordering) { - // Reverse requirement is satisfied by exiting ordering. - // Hence reverse the aggregator - *requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - return Ok(()); + // Requirement is satisfied by existing ordering + requirement = finer_ordering; + continue; } } - } - if let Some(finer_ordering) = - finer_ordering(requirement, aggr_expr, group_by, eq_properties, agg_mode) - { - // There is a requirement that both satisfies existing requirement and current - // aggregate requirement. Use updated requirement - *requirement = finer_ordering; - return Ok(()); - } - if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - requirement, - &reverse_aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - // There is a requirement that both satisfies existing requirement and reverse + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + if eq_properties.ordering_satisfy(&finer_ordering) { + // Reverse requirement is satisfied by exiting ordering. + // Hence reverse the aggregator + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + } + if let Some(finer_ordering) = + finer_ordering(&requirement, aggr_expr, group_by, eq_properties, agg_mode) + { + // There is a requirement that both satisfies existing requirement and current // aggregate requirement. Use updated requirement - *requirement = finer_ordering; - *aggr_expr = reverse_aggr_expr; - return Ok(()); + requirement = finer_ordering; + continue; } + if let Some(reverse_aggr_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &requirement, + &reverse_aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + // There is a requirement that both satisfies existing requirement and reverse + // aggregate requirement. Use updated requirement + requirement = finer_ordering; + *aggr_expr = reverse_aggr_expr; + continue; + } + } + + // Neither the existing requirement and current aggregate requirement satisfy the other, this means + // requirements are conflicting. Currently, we do not support + // conflicting requirements. + return not_impl_err!( + "Conflicting ordering requirements in aggregate functions is not supported" + ) } - // Neither the existing requirement and current aggregate requirement satisfy the other, this means - // requirements are conflicting. Currently, we do not support - // conflicting requirements. - not_impl_err!( - "Conflicting ordering requirements in aggregate functions is not supported" - ) + Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) } /// returns physical expressions for arguments to evaluate against a batch @@ -2173,7 +2151,7 @@ mod tests { }) .collect::>(); let group_by = PhysicalGroupBy::new_single(vec![]); - let res = get_aggregate_exprs_requirement( + let res = get_finer_aggregate_exprs_requirement( &mut aggr_exprs, &group_by, &eq_properties, From d3daa4e025ecf97671bd3e93ec55d4599fbda84a Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 15:45:07 +0800 Subject: [PATCH 30/77] fmt Signed-off-by: jayzhan211 --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e24e71e73540..ba9a6b1be0ef 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -987,7 +987,7 @@ fn get_finer_aggregate_exprs_requirement( // conflicting requirements. return not_impl_err!( "Conflicting ordering requirements in aggregate functions is not supported" - ) + ); } Ok(PhysicalSortRequirement::from_sort_exprs(&requirement)) From dcb3e90f630b971b57c9e19799e94109ba8f18f3 Mon Sep 17 00:00:00 2001 From: jayzhan211 Date: Sat, 13 Apr 2024 22:04:31 +0800 Subject: [PATCH 31/77] avoid unnecessary recursion and rename Signed-off-by: jayzhan211 --- .../physical_optimizer/convert_first_last.rs | 112 ++++++------------ .../core/src/physical_optimizer/optimizer.rs | 6 +- .../sqllogictest/test_files/explain.slt | 12 +- 3 files changed, 48 insertions(+), 82 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 45e95827c773..4102313d3126 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -46,15 +46,15 @@ use super::PhysicalOptimizerRule; /// so we can convert the aggregate expression to FirstValue(c1 order by asc), /// since the current ordering is already satisfied, it saves our time! #[derive(Default)] -pub struct ConvertFirstLast {} +pub struct OptimizeAggregateOrder {} -impl ConvertFirstLast { +impl OptimizeAggregateOrder { pub fn new() -> Self { Self::default() } } -impl PhysicalOptimizerRule for ConvertFirstLast { +impl PhysicalOptimizerRule for OptimizeAggregateOrder { fn optimize( &self, plan: Arc, @@ -65,7 +65,7 @@ impl PhysicalOptimizerRule for ConvertFirstLast { } fn name(&self) -> &str { - "SimpleOrdering" + "OptimizeAggregateOrder" } fn schema_check(&self) -> bool { @@ -75,75 +75,6 @@ impl PhysicalOptimizerRule for ConvertFirstLast { fn get_common_requirement_of_aggregate_input( plan: Arc, -) -> Result>> { - // Optimize children - let children = plan.children(); - let mut is_child_transformed = false; - let mut new_children: Vec> = vec![]; - for c in children.iter() { - let res = optimize_internal(c.clone())?; - if res.transformed { - is_child_transformed = true; - } - new_children.push(res.data); - } - - // Update children if transformed - let plan = if is_child_transformed { - plan.with_new_children(new_children)? - } else { - plan - }; - - // Update itself - let plan = optimize_internal(plan)?; - - // If one of the children is transformed, then the plan is considered transformed, then we update - // the children of the plan from bottom to top. - if plan.transformed || is_child_transformed { - Ok(Transformed::yes(plan.data)) - } else { - Ok(Transformed::no(plan.data)) - } -} - -/// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer -/// is in Partial mode and the second layer is in Final or Finalpartitioned mode. -/// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. -/// Therefore, we check it and get the updated aggregate expressions. -/// -/// If AggregateExec is created from elsewhere, we skip the check and return the original aggregate expressions. -fn try_get_updated_aggr_expr_from_child( - aggr_exec: &AggregateExec, -) -> Vec> { - let input = aggr_exec.input(); - if aggr_exec.mode() == &AggregateMode::Final - || aggr_exec.mode() == &AggregateMode::FinalPartitioned - { - // Some aggregators may be modified during initialization for - // optimization purposes. For example, a FIRST_VALUE may turn - // into a LAST_VALUE with the reverse ordering requirement. - // To reflect such changes to subsequent stages, use the updated - // `AggregateExpr`/`PhysicalSortExpr` objects. - // - // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] - if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { - if c_aggr_exec.mode() == &AggregateMode::Partial { - // If the input is an AggregateExec in Partial mode, then the - // input is a CoalescePartitionsExec. In this case, the - // AggregateExec is the second stage of aggregation. The - // requirements of the second stage are the requirements of - // the first stage. - return c_aggr_exec.aggr_expr().to_vec(); - } - } - } - - aggr_exec.aggr_expr().to_vec() -} - -fn optimize_internal( - plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { let input = aggr_exec.input(); @@ -207,6 +138,41 @@ fn optimize_internal( } } +/// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer +/// is in Partial mode and the second layer is in Final or Finalpartitioned mode. +/// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. +/// Therefore, we check it and get the updated aggregate expressions. +/// +/// If AggregateExec is created from elsewhere, we skip the check and return the original aggregate expressions. +fn try_get_updated_aggr_expr_from_child( + aggr_exec: &AggregateExec, +) -> Vec> { + let input = aggr_exec.input(); + if aggr_exec.mode() == &AggregateMode::Final + || aggr_exec.mode() == &AggregateMode::FinalPartitioned + { + // Some aggregators may be modified during initialization for + // optimization purposes. For example, a FIRST_VALUE may turn + // into a LAST_VALUE with the reverse ordering requirement. + // To reflect such changes to subsequent stages, use the updated + // `AggregateExpr`/`PhysicalSortExpr` objects. + // + // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] + if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { + if c_aggr_exec.mode() == &AggregateMode::Partial { + // If the input is an AggregateExec in Partial mode, then the + // input is a CoalescePartitionsExec. In this case, the + // AggregateExec is the second stage of aggregation. The + // requirements of the second stage are the requirements of + // the first stage. + return c_aggr_exec.aggr_expr().to_vec(); + } + } + } + + aggr_exec.aggr_expr().to_vec() +} + /// Get the common requirement that satisfies all the aggregate expressions. /// /// # Parameters diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index e8ae03e11c62..70ac12344810 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,7 +19,7 @@ use std::sync::Arc; -use super::convert_first_last::ConvertFirstLast; +use super::convert_first_last::OptimizeAggregateOrder; use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; @@ -91,7 +91,7 @@ impl PhysicalOptimizer { // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), // Run once before PartialFinalAggregation is rewritten to ensure the rule is applied correctly - Arc::new(ConvertFirstLast::new()), + Arc::new(OptimizeAggregateOrder::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at @@ -105,7 +105,7 @@ impl PhysicalOptimizer { // as the latter may break local sorting requirements. Arc::new(EnforceSorting::new()), // Run once after the local sorting requirement is changed - Arc::new(ConvertFirstLast::new()), + Arc::new(OptimizeAggregateOrder::new()), // TODO: `try_embed_to_hash_join` in the ProjectionPushdown rule would be block by the CoalesceBatches, so add it before CoalesceBatches. Maybe optimize it in the future. Arc::new(ProjectionPushdown::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 3ec626aed698..467bbca3c0bf 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -245,11 +245,11 @@ OutputRequirementExec physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.csv]]}, projection=[a, b, c], has_header=true @@ -303,11 +303,11 @@ OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[ physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements @@ -341,11 +341,11 @@ OutputRequirementExec physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE -physical_plan after SimpleOrdering SAME TEXT AS ABOVE +physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after ProjectionPushdown SAME TEXT AS ABOVE physical_plan after coalesce_batches SAME TEXT AS ABOVE physical_plan after OutputRequirements From 4983fee4f354fd1a26955137c75f5fed09417336 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 15:05:30 +0300 Subject: [PATCH 32/77] Minor changes --- .../physical_optimizer/convert_first_last.rs | 29 +------------------ .../core/src/physical_optimizer/optimizer.rs | 2 -- .../physical-plan/src/aggregates/mod.rs | 9 ++---- .../sqllogictest/test_files/aggregate.slt | 4 +-- .../sqllogictest/test_files/explain.slt | 3 -- .../sqllogictest/test_files/group_by.slt | 4 +-- 6 files changed, 8 insertions(+), 43 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 4102313d3126..f555c94ed40d 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -80,7 +80,6 @@ fn get_common_requirement_of_aggregate_input( let input = aggr_exec.input(); let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); let group_by = aggr_exec.group_by(); - let mode = aggr_exec.mode(); let input_eq_properties = input.equivalence_properties(); let groupby_exprs = group_by.input_exprs(); @@ -102,33 +101,7 @@ fn get_common_requirement_of_aggregate_input( input_eq_properties, )?; - let required_input_ordering = (!requirement.is_empty()).then_some(requirement); - - let input_order_mode = - if indices.len() == groupby_exprs.len() && !indices.is_empty() { - InputOrderMode::Sorted - } else if !indices.is_empty() { - InputOrderMode::PartiallySorted(indices) - } else { - InputOrderMode::Linear - }; - let projection_mapping = - ProjectionMapping::try_new(group_by.expr(), &input.schema())?; - - let cache = AggregateExec::compute_properties( - input, - plan.schema().clone(), - &projection_mapping, - mode, - &input_order_mode, - ); - - let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info( - required_input_ordering, - aggr_expr, - cache, - input_order_mode, - ); + let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info(aggr_expr); Ok(Transformed::yes( Arc::new(aggr_exec) as Arc diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 70ac12344810..08cbf68fa617 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -90,8 +90,6 @@ impl PhysicalOptimizer { // as that rule may inject other operations in between the different AggregateExecs. // Applying the rule early means only directly-connected AggregateExecs must be examined. Arc::new(LimitedDistinctAggregation::new()), - // Run once before PartialFinalAggregation is rewritten to ensure the rule is applied correctly - Arc::new(OptimizeAggregateOrder::new()), // The EnforceDistribution rule is for adding essential repartitioning to satisfy distribution // requirements. Please make sure that the whole plan tree is determined before this rule. // This rule increases parallelism if doing so is beneficial to the physical plan; i.e. at diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ba9a6b1be0ef..3a7d7005654a 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -276,17 +276,14 @@ impl AggregateExec { /// where we need parts of the new value, others cloned from the old one pub fn new_with_aggr_expr_and_ordering_info( &self, - required_input_ordering: Option, aggr_expr: Vec>, - cache: PlanProperties, - input_order_mode: InputOrderMode, ) -> Self { Self { aggr_expr, - required_input_ordering, + required_input_ordering: self.required_input_ordering.clone(), metrics: ExecutionPlanMetricsSet::new(), - input_order_mode, - cache, + input_order_mode: self.input_order_mode.clone(), + cache: self.cache.clone(), // clone the rest of the fields mode: self.mode, group_by: self.group_by.clone(), diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 69fb4ea38e9f..3b044d980a91 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3451,7 +3451,7 @@ logical_plan Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] --TableScan: convert_first_last_table projection=[c1, c3] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 @@ -3465,7 +3465,7 @@ logical_plan Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] --TableScan: convert_first_last_table projection=[c1, c2] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] --CoalescePartitionsExec ----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] ------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index 467bbca3c0bf..0e5195388c83 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -245,7 +245,6 @@ OutputRequirementExec physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE @@ -303,7 +302,6 @@ OutputRequirementExec, statistics=[Rows=Exact(8), Bytes=Absent, [(Col[0]:),(Col[ physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE @@ -341,7 +339,6 @@ OutputRequirementExec physical_plan after aggregate_statistics SAME TEXT AS ABOVE physical_plan after join_selection SAME TEXT AS ABOVE physical_plan after LimitedDistinctAggregation SAME TEXT AS ABOVE -physical_plan after OptimizeAggregateOrder SAME TEXT AS ABOVE physical_plan after EnforceDistribution SAME TEXT AS ABOVE physical_plan after CombinePartialFinalAggregate SAME TEXT AS ABOVE physical_plan after EnforceSorting SAME TEXT AS ABOVE diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 869462b4722a..c15879c75636 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2805,7 +2805,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal ------TableScan: sales_global projection=[country, ts, amount] physical_plan ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), SUM(sales_global.amount)] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] ----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -3800,7 +3800,7 @@ Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_tab ----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] ----CoalesceBatchesExec: target_batch_size=2 ------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 --------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] From 4f0105d5563ab237bdcf898d7374d9f73b8aac26 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 15:55:49 +0300 Subject: [PATCH 33/77] Add new API for aggregate optimization --- .../physical_optimizer/convert_first_last.rs | 99 ++++--------------- .../functions-aggregate/src/first_last.rs | 26 +++++ .../physical-expr-common/src/aggregate/mod.rs | 19 +++- 3 files changed, 61 insertions(+), 83 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index f555c94ed40d..2753d00ad3de 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -20,15 +20,12 @@ use datafusion_common::{ config::ConfigOptions, tree_node::{Transformed, TransformedResult, TreeNode}, }; -use datafusion_physical_expr::expressions::{FirstValue, LastValue}; use datafusion_physical_expr::{ - equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, - EquivalenceProperties, PhysicalSortRequirement, + reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalSortRequirement, }; use datafusion_physical_plan::aggregates::concat_slices; use datafusion_physical_plan::{ - aggregates::{AggregateExec, AggregateMode}, - ExecutionPlan, ExecutionPlanProperties, InputOrderMode, + aggregates::AggregateExec, ExecutionPlan, ExecutionPlanProperties, }; use std::sync::Arc; @@ -77,8 +74,11 @@ fn get_common_requirement_of_aggregate_input( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + if !aggr_exec.mode().is_first_stage() { + return Ok(Transformed::no(plan)); + } let input = aggr_exec.input(); - let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); + let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); let group_by = aggr_exec.group_by(); let input_eq_properties = input.equivalence_properties(); @@ -95,11 +95,7 @@ fn get_common_requirement_of_aggregate_input( }) .collect::>(); - try_convert_first_last_if_better( - &requirement, - &mut aggr_expr, - input_eq_properties, - )?; + try_convert_reverse_if_better(&requirement, &mut aggr_expr, input_eq_properties)?; let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info(aggr_expr); @@ -111,41 +107,6 @@ fn get_common_requirement_of_aggregate_input( } } -/// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer -/// is in Partial mode and the second layer is in Final or Finalpartitioned mode. -/// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. -/// Therefore, we check it and get the updated aggregate expressions. -/// -/// If AggregateExec is created from elsewhere, we skip the check and return the original aggregate expressions. -fn try_get_updated_aggr_expr_from_child( - aggr_exec: &AggregateExec, -) -> Vec> { - let input = aggr_exec.input(); - if aggr_exec.mode() == &AggregateMode::Final - || aggr_exec.mode() == &AggregateMode::FinalPartitioned - { - // Some aggregators may be modified during initialization for - // optimization purposes. For example, a FIRST_VALUE may turn - // into a LAST_VALUE with the reverse ordering requirement. - // To reflect such changes to subsequent stages, use the updated - // `AggregateExpr`/`PhysicalSortExpr` objects. - // - // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] - if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { - if c_aggr_exec.mode() == &AggregateMode::Partial { - // If the input is an AggregateExec in Partial mode, then the - // input is a CoalescePartitionsExec. In this case, the - // AggregateExec is the second stage of aggregation. The - // requirements of the second stage are the requirements of - // the first stage. - return c_aggr_exec.aggr_expr().to_vec(); - } - } - } - - aggr_exec.aggr_expr().to_vec() -} - /// Get the common requirement that satisfies all the aggregate expressions. /// /// # Parameters @@ -166,7 +127,7 @@ fn try_get_updated_aggr_expr_from_child( /// /// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this /// function care only the possible conversion between FIRST_VALUE and LAST_VALUE -fn try_convert_first_last_if_better( +fn try_convert_reverse_if_better( prefix_requirement: &[PhysicalSortRequirement], aggr_exprs: &mut [Arc], eq_properties: &EquivalenceProperties, @@ -178,54 +139,28 @@ fn try_convert_first_last_if_better( let reverse_aggr_req = PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { - let mut first_value = first_value.clone(); - + if !aggr_expr.is_order_sensitive() && !aggr_req.is_empty() { if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &aggr_req, )) { - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &reverse_aggr_req, )) { // Converting to LAST_VALUE enables more efficient execution // given the existing ordering: - let mut last_value = first_value.convert_to_last(); - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else { + if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { + *aggr_expr = aggr_expr_rev; + } else { + continue; + } + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; // Requirement is not satisfied with existing ordering. - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; - } - continue; - } - if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { - let mut last_value = last_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to FIRST_VALUE enables more efficient execution - // given the existing ordering: - let mut first_value = last_value.convert_to_first(); - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; } else { - // Requirement is not satisfied with existing ordering. - last_value = last_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(last_value) as _; + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(false)?; } - continue; } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index d5367ad34163..394b945a391b 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -479,6 +479,19 @@ impl AggregateExpr for FirstValuePhysicalExpr { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } + fn is_order_sensitive(&self) -> bool { + false + } + + fn with_requirement_satisfied( + self: Arc, + requirement_satisfied: bool, + ) -> Result> { + let mut res = self.as_ref().clone(); + res.requirement_satisfied = requirement_satisfied; + Ok(Arc::new(res)) + } + fn name(&self) -> &str { &self.name } @@ -654,6 +667,19 @@ impl AggregateExpr for LastValuePhysicalExpr { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } + fn is_order_sensitive(&self) -> bool { + false + } + + fn with_requirement_satisfied( + self: Arc, + requirement_satisfied: bool, + ) -> Result> { + let mut res = self.as_ref().clone(); + res.requirement_satisfied = requirement_satisfied; + Ok(Arc::new(res)) + } + fn name(&self) -> &str { &self.name } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 33044fd9beee..a110aadf5081 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -18,7 +18,7 @@ pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; -use datafusion_common::{not_impl_err, Result}; +use datafusion_common::{exec_datafusion_err, not_impl_err, Result}; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; @@ -103,6 +103,23 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { None } + fn is_order_sensitive(&self) -> bool { + true + } + + fn with_requirement_satisfied( + self: Arc, + _requirement_satisfied: bool, + ) -> Result> { + if self.order_bys().is_some() && !self.is_order_sensitive() { + return Err(exec_datafusion_err!( + "Should implement with satisfied for aggregator :{:?}", + self.name() + )); + } + unreachable!() + } + /// Human readable name such as `"MIN(c2)"`. The default /// implementation returns placeholder text. fn name(&self) -> &str { From 4b1e24f5009a6ecb9e264a54de29b0f34a51203b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 16:00:04 +0300 Subject: [PATCH 34/77] Minor changes --- .../core/src/physical_optimizer/convert_first_last.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs index 2753d00ad3de..3e7f440c7e06 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_first_last.rs @@ -113,12 +113,8 @@ fn get_common_requirement_of_aggregate_input( /// /// - `aggr_exprs`: A slice of `Arc` containing all the /// aggregate expressions. -/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the -/// physical GROUP BY expression. /// - `eq_properties`: A reference to an `EquivalenceProperties` instance /// representing equivalence properties for ordering. -/// - `agg_mode`: A reference to an `AggregateMode` instance representing the -/// mode of aggregation. /// /// # Returns /// @@ -149,11 +145,12 @@ fn try_convert_reverse_if_better( prefix_requirement, &reverse_aggr_req, )) { - // Converting to LAST_VALUE enables more efficient execution + // Converting to reverse enables more efficient execution // given the existing ordering: if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { *aggr_expr = aggr_expr_rev; } else { + // If reverse execution is not possible, cannot update current aggregate expression. continue; } *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; From d95d6ded736629804754f747ff15af477137c30c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 16:17:01 +0300 Subject: [PATCH 35/77] Minor changes --- ...nvert_first_last.rs => convert_reverse.rs} | 53 ++++++++++--------- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../core/src/physical_optimizer/optimizer.rs | 2 +- .../physical-expr-common/src/aggregate/mod.rs | 4 ++ .../physical-plan/src/aggregates/mod.rs | 8 ++- 5 files changed, 38 insertions(+), 31 deletions(-) rename datafusion/core/src/physical_optimizer/{convert_first_last.rs => convert_reverse.rs} (74%) diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_reverse.rs similarity index 74% rename from datafusion/core/src/physical_optimizer/convert_first_last.rs rename to datafusion/core/src/physical_optimizer/convert_reverse.rs index 3e7f440c7e06..53859e672372 100644 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ b/datafusion/core/src/physical_optimizer/convert_reverse.rs @@ -57,7 +57,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { plan: Arc, _config: &ConfigOptions, ) -> Result> { - plan.transform_up(&get_common_requirement_of_aggregate_input) + plan.transform_up(&replace_with_reverse_aggregator_when_beneficial) .data() } @@ -70,7 +70,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { } } -fn get_common_requirement_of_aggregate_input( +fn replace_with_reverse_aggregator_when_beneficial( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { @@ -97,7 +97,7 @@ fn get_common_requirement_of_aggregate_input( try_convert_reverse_if_better(&requirement, &mut aggr_expr, input_eq_properties)?; - let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info(aggr_expr); + let aggr_exec = aggr_exec.with_new_aggr_exprs(aggr_expr); Ok(Transformed::yes( Arc::new(aggr_exec) as Arc @@ -135,29 +135,34 @@ fn try_convert_reverse_if_better( let reverse_aggr_req = PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - if !aggr_expr.is_order_sensitive() && !aggr_req.is_empty() { - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to reverse enables more efficient execution - // given the existing ordering: - if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { - *aggr_expr = aggr_expr_rev; - } else { - // If reverse execution is not possible, cannot update current aggregate expression. - continue; - } - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; - // Requirement is not satisfied with existing ordering. + if aggr_expr.is_order_sensitive() || aggr_req.is_empty() { + // If ordering for the aggregator is absolute requirement + // or there is no requirement for the aggregator, shouldn't update the aggregator + continue; + } + if eq_properties + .ordering_satisfy_requirement(&concat_slices(prefix_requirement, &aggr_req)) + { + // Existing ordering satisfy the requirement of the aggregator + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to reverse enables more efficient execution + // given the existing ordering: + if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { + *aggr_expr = aggr_expr_rev; } else { - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(false)?; + // If reverse execution is not possible, cannot update current aggregate expression. + continue; } + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; + // Requirement is not satisfied with existing ordering. + } else { + // Requirement is not satisfied for the aggregator (Please note that: Aggregator can still work in this case, guaranteed by order sensitive flag being false. + // However, It will be inefficient compared to version where requirement is satisfied). + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(false)?; } } diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index c80668c6da74..cf8f6860176b 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -24,7 +24,7 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; -mod convert_first_last; +mod convert_reverse; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 08cbf68fa617..8a7447bfe932 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,7 +19,7 @@ use std::sync::Arc; -use super::convert_first_last::OptimizeAggregateOrder; +use super::convert_reverse::OptimizeAggregateOrder; use super::projection_pushdown::ProjectionPushdown; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index a110aadf5081..11f3edcb3ad1 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -103,10 +103,14 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { None } + /// Indicates whether aggregator can produce correct result with any arbitrary ordering or not. fn is_order_sensitive(&self) -> bool { true } + /// Indicates whether requirement of the aggregators is satisfied at the input. + /// If this is not the case some order in-sensitive aggregators can still produce + /// correct result with possibly more work internally. fn with_requirement_satisfied( self: Arc, _requirement_satisfied: bool, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3a7d7005654a..e1f97b665d55 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -274,17 +274,15 @@ pub struct AggregateExec { impl AggregateExec { /// Function used in `ConvertFirstLast` optimizer rule, /// where we need parts of the new value, others cloned from the old one - pub fn new_with_aggr_expr_and_ordering_info( - &self, - aggr_expr: Vec>, - ) -> Self { + /// Rewrites aggregate exec with new aggregate expressions. + pub fn with_new_aggr_exprs(&self, aggr_expr: Vec>) -> Self { Self { aggr_expr, + // clone the rest of the fields required_input_ordering: self.required_input_ordering.clone(), metrics: ExecutionPlanMetricsSet::new(), input_order_mode: self.input_order_mode.clone(), cache: self.cache.clone(), - // clone the rest of the fields mode: self.mode, group_by: self.group_by.clone(), filter_expr: self.filter_expr.clone(), From 25119bd8d682668f55ef5155b9f0f69ab566bf9b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 16:23:16 +0300 Subject: [PATCH 36/77] Remove old code --- .../physical_optimizer/convert_first_last.rs | 260 ------------------ 1 file changed, 260 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/convert_first_last.rs diff --git a/datafusion/core/src/physical_optimizer/convert_first_last.rs b/datafusion/core/src/physical_optimizer/convert_first_last.rs deleted file mode 100644 index 4102313d3126..000000000000 --- a/datafusion/core/src/physical_optimizer/convert_first_last.rs +++ /dev/null @@ -1,260 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -use datafusion_common::Result; -use datafusion_common::{ - config::ConfigOptions, - tree_node::{Transformed, TransformedResult, TreeNode}, -}; -use datafusion_physical_expr::expressions::{FirstValue, LastValue}; -use datafusion_physical_expr::{ - equivalence::ProjectionMapping, reverse_order_bys, AggregateExpr, - EquivalenceProperties, PhysicalSortRequirement, -}; -use datafusion_physical_plan::aggregates::concat_slices; -use datafusion_physical_plan::{ - aggregates::{AggregateExec, AggregateMode}, - ExecutionPlan, ExecutionPlanProperties, InputOrderMode, -}; -use std::sync::Arc; - -use datafusion_physical_plan::windows::get_ordered_partition_by_indices; - -use super::PhysicalOptimizerRule; - -/// The optimizer rule check the ordering requirements of the aggregate expressions. -/// And convert between FIRST_VALUE and LAST_VALUE if possible. -/// For example, If we have an ascending values and we want LastValue from the descending requirement, -/// it is equivalent to FirstValue with the current ascending ordering. -/// -/// The concrete example is that, says we have values c1 with [1, 2, 3], which is an ascending order. -/// If we want LastValue(c1 order by desc), which is the first value of reversed c1 [3, 2, 1], -/// so we can convert the aggregate expression to FirstValue(c1 order by asc), -/// since the current ordering is already satisfied, it saves our time! -#[derive(Default)] -pub struct OptimizeAggregateOrder {} - -impl OptimizeAggregateOrder { - pub fn new() -> Self { - Self::default() - } -} - -impl PhysicalOptimizerRule for OptimizeAggregateOrder { - fn optimize( - &self, - plan: Arc, - _config: &ConfigOptions, - ) -> Result> { - plan.transform_up(&get_common_requirement_of_aggregate_input) - .data() - } - - fn name(&self) -> &str { - "OptimizeAggregateOrder" - } - - fn schema_check(&self) -> bool { - true - } -} - -fn get_common_requirement_of_aggregate_input( - plan: Arc, -) -> Result>> { - if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - let input = aggr_exec.input(); - let mut aggr_expr = try_get_updated_aggr_expr_from_child(aggr_exec); - let group_by = aggr_exec.group_by(); - let mode = aggr_exec.mode(); - - let input_eq_properties = input.equivalence_properties(); - let groupby_exprs = group_by.input_exprs(); - // If existing ordering satisfies a prefix of the GROUP BY expressions, - // prefix requirements with this section. In this case, aggregation will - // work more efficiently. - let indices = get_ordered_partition_by_indices(&groupby_exprs, input); - let requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - - try_convert_first_last_if_better( - &requirement, - &mut aggr_expr, - input_eq_properties, - )?; - - let required_input_ordering = (!requirement.is_empty()).then_some(requirement); - - let input_order_mode = - if indices.len() == groupby_exprs.len() && !indices.is_empty() { - InputOrderMode::Sorted - } else if !indices.is_empty() { - InputOrderMode::PartiallySorted(indices) - } else { - InputOrderMode::Linear - }; - let projection_mapping = - ProjectionMapping::try_new(group_by.expr(), &input.schema())?; - - let cache = AggregateExec::compute_properties( - input, - plan.schema().clone(), - &projection_mapping, - mode, - &input_order_mode, - ); - - let aggr_exec = aggr_exec.new_with_aggr_expr_and_ordering_info( - required_input_ordering, - aggr_expr, - cache, - input_order_mode, - ); - - Ok(Transformed::yes( - Arc::new(aggr_exec) as Arc - )) - } else { - Ok(Transformed::no(plan)) - } -} - -/// In `create_initial_plan` for LogicalPlan::Aggregate, we have a nested AggregateExec where the first layer -/// is in Partial mode and the second layer is in Final or Finalpartitioned mode. -/// If the first layer of aggregate plan is transformed, we need to update the child of the layer with final mode. -/// Therefore, we check it and get the updated aggregate expressions. -/// -/// If AggregateExec is created from elsewhere, we skip the check and return the original aggregate expressions. -fn try_get_updated_aggr_expr_from_child( - aggr_exec: &AggregateExec, -) -> Vec> { - let input = aggr_exec.input(); - if aggr_exec.mode() == &AggregateMode::Final - || aggr_exec.mode() == &AggregateMode::FinalPartitioned - { - // Some aggregators may be modified during initialization for - // optimization purposes. For example, a FIRST_VALUE may turn - // into a LAST_VALUE with the reverse ordering requirement. - // To reflect such changes to subsequent stages, use the updated - // `AggregateExpr`/`PhysicalSortExpr` objects. - // - // The bottom up transformation is the mirror of LogicalPlan::Aggregate creation in [create_initial_plan] - if let Some(c_aggr_exec) = input.as_any().downcast_ref::() { - if c_aggr_exec.mode() == &AggregateMode::Partial { - // If the input is an AggregateExec in Partial mode, then the - // input is a CoalescePartitionsExec. In this case, the - // AggregateExec is the second stage of aggregation. The - // requirements of the second stage are the requirements of - // the first stage. - return c_aggr_exec.aggr_expr().to_vec(); - } - } - } - - aggr_exec.aggr_expr().to_vec() -} - -/// Get the common requirement that satisfies all the aggregate expressions. -/// -/// # Parameters -/// -/// - `aggr_exprs`: A slice of `Arc` containing all the -/// aggregate expressions. -/// - `group_by`: A reference to a `PhysicalGroupBy` instance representing the -/// physical GROUP BY expression. -/// - `eq_properties`: A reference to an `EquivalenceProperties` instance -/// representing equivalence properties for ordering. -/// - `agg_mode`: A reference to an `AggregateMode` instance representing the -/// mode of aggregation. -/// -/// # Returns -/// -/// A `LexRequirement` instance, which is the requirement that satisfies all the -/// aggregate requirements. Returns an error in case of conflicting requirements. -/// -/// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this -/// function care only the possible conversion between FIRST_VALUE and LAST_VALUE -fn try_convert_first_last_if_better( - prefix_requirement: &[PhysicalSortRequirement], - aggr_exprs: &mut [Arc], - eq_properties: &EquivalenceProperties, -) -> Result<()> { - for aggr_expr in aggr_exprs.iter_mut() { - let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_req = reverse_order_bys(aggr_req); - let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - - if let Some(first_value) = aggr_expr.as_any().downcast_ref::() { - let mut first_value = first_value.clone(); - - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to LAST_VALUE enables more efficient execution - // given the existing ordering: - let mut last_value = first_value.convert_to_last(); - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - first_value = first_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(first_value) as _; - } - continue; - } - if let Some(last_value) = aggr_expr.as_any().downcast_ref::() { - let mut last_value = last_value.clone(); - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - last_value = last_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(last_value) as _; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to FIRST_VALUE enables more efficient execution - // given the existing ordering: - let mut first_value = last_value.convert_to_first(); - first_value = first_value.with_requirement_satisfied(true); - *aggr_expr = Arc::new(first_value) as _; - } else { - // Requirement is not satisfied with existing ordering. - last_value = last_value.with_requirement_satisfied(false); - *aggr_expr = Arc::new(last_value) as _; - } - continue; - } - } - - Ok(()) -} From 61f5b43c7bdd21b8317424d7311c65028a2722a3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 16:56:20 +0300 Subject: [PATCH 37/77] Minor changes --- .../src/physical_optimizer/convert_reverse.rs | 2 +- .../functions-aggregate/src/first_last.rs | 12 ++++---- .../physical-expr-common/src/aggregate/mod.rs | 29 +++++++++++++++++-- .../src/aggregate/array_agg_ordered.rs | 5 ++++ .../physical-expr/src/aggregate/nth_value.rs | 5 ++++ .../sqllogictest/test_files/aggregate.slt | 28 +++++++++--------- .../sqllogictest/test_files/group_by.slt | 20 ++++++------- 7 files changed, 68 insertions(+), 33 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_reverse.rs b/datafusion/core/src/physical_optimizer/convert_reverse.rs index 53859e672372..434da1578f7b 100644 --- a/datafusion/core/src/physical_optimizer/convert_reverse.rs +++ b/datafusion/core/src/physical_optimizer/convert_reverse.rs @@ -135,7 +135,7 @@ fn try_convert_reverse_if_better( let reverse_aggr_req = PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - if aggr_expr.is_order_sensitive() || aggr_req.is_empty() { + if aggr_expr.order_sensitivity().is_order_hard_required() || aggr_req.is_empty() { // If ordering for the aggregator is absolute requirement // or there is no requirement for the aggregator, shouldn't update the aggregator continue; diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 394b945a391b..055834543c67 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -31,7 +31,9 @@ use datafusion_expr::{Accumulator, AggregateUDFImpl, Expr, Signature, Volatility use datafusion_physical_expr_common::aggregate::utils::{ down_cast_any_ref, get_sort_options, ordering_fields, }; -use datafusion_physical_expr_common::aggregate::AggregateExpr; +use datafusion_physical_expr_common::aggregate::{ + AggregateExpr, AggregateOrderSensitivity, +}; use datafusion_physical_expr_common::expressions; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; @@ -479,8 +481,8 @@ impl AggregateExpr for FirstValuePhysicalExpr { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } - fn is_order_sensitive(&self) -> bool { - false + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::OrderBeneficial } fn with_requirement_satisfied( @@ -667,8 +669,8 @@ impl AggregateExpr for LastValuePhysicalExpr { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } - fn is_order_sensitive(&self) -> bool { - false + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::OrderBeneficial } fn with_requirement_satisfied( diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 11f3edcb3ad1..7c57a56720e9 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -66,6 +66,24 @@ pub fn create_aggregate_expr( })) } +/// Defines how input ordering effects the aggregator +#[derive(Debug, Eq, PartialEq, Clone, Copy)] +pub enum AggregateOrderSensitivity { + OrderInsensitive, + OrderRequiring, + OrderBeneficial, +} + +impl AggregateOrderSensitivity { + pub fn is_order_beneficial(&self) -> bool { + self.eq(&AggregateOrderSensitivity::OrderBeneficial) + } + + pub fn is_order_hard_required(&self) -> bool { + self.eq(&AggregateOrderSensitivity::OrderRequiring) + } +} + /// An aggregate expression that: /// * knows its resulting field /// * knows how to create its accumulator @@ -104,8 +122,8 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { } /// Indicates whether aggregator can produce correct result with any arbitrary ordering or not. - fn is_order_sensitive(&self) -> bool { - true + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::OrderInsensitive } /// Indicates whether requirement of the aggregators is satisfied at the input. @@ -115,12 +133,17 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { self: Arc, _requirement_satisfied: bool, ) -> Result> { - if self.order_bys().is_some() && !self.is_order_sensitive() { + if self.order_bys().is_some() && self.order_sensitivity().is_order_beneficial() { return Err(exec_datafusion_err!( "Should implement with satisfied for aggregator :{:?}", self.name() )); } + println!( + "self.order_sensitivity(): {:?}, aggregator :{:?}", + self.order_sensitivity(), + self.name() + ); unreachable!() } diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 7e2c7bb27144..b64dd22bf4b9 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -40,6 +40,7 @@ use datafusion_common::utils::array_into_list_array; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::Accumulator; +use datafusion_physical_expr_common::aggregate::AggregateOrderSensitivity; /// Expression for a `ARRAY_AGG(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, @@ -131,6 +132,10 @@ impl AggregateExpr for OrderSensitiveArrayAgg { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::OrderRequiring + } + fn name(&self) -> &str { &self.name } diff --git a/datafusion/physical-expr/src/aggregate/nth_value.rs b/datafusion/physical-expr/src/aggregate/nth_value.rs index dba259a507fd..846c8bf785bb 100644 --- a/datafusion/physical-expr/src/aggregate/nth_value.rs +++ b/datafusion/physical-expr/src/aggregate/nth_value.rs @@ -35,6 +35,7 @@ use arrow_schema::{DataType, Field, Fields}; use datafusion_common::utils::{array_into_list_array, get_row_at_idx}; use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; use datafusion_expr::Accumulator; +use datafusion_physical_expr_common::aggregate::AggregateOrderSensitivity; /// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, @@ -125,6 +126,10 @@ impl AggregateExpr for NthValueAgg { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::OrderRequiring + } + fn name(&self) -> &str { &self.name } diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index e11b4ef370dc..3d24fe3888d7 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3475,25 +3475,25 @@ query TT explain select first_value(c1 order by c3 desc) from convert_first_last_table; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] ---TableScan: convert_first_last_table projection=[c1, c3] +01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] +02)--TableScan: convert_first_last_table projection=[c1, c3] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true # test last to first query TT explain select last_value(c1 order by c2 asc) from convert_first_last_table; ---- logical_plan -Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] ---TableScan: convert_first_last_table projection=[c1, c2] +01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] +02)--TableScan: convert_first_last_table projection=[c1, c2] physical_plan -AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] ---CoalescePartitionsExec -----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] -------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1)] +02)--CoalescePartitionsExec +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1)] +04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index 3e88568170e5..5c5bf58dd049 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2804,9 +2804,9 @@ logical_plan 03)----Sort: sales_global.ts ASC NULLS LAST 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan -ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] -----MemoryExec: partitions=1, partition_sizes=[1] +01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), SUM(sales_global.amount)] +03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -3799,13 +3799,13 @@ logical_plan 02)--Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] 03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan -ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] -----CoalesceBatchesExec: target_batch_size=2 -------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 ---------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true +01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), LAST_VALUE(multiple_ordered_table.c)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 +05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, From 857eedb528108ed0d071942f4026d46845d6f7ee Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 17:02:24 +0300 Subject: [PATCH 38/77] Minor changes --- datafusion/functions-aggregate/src/first_last.rs | 4 ++-- .../physical-expr-common/src/aggregate/mod.rs | 16 ++++++++++------ .../src/aggregate/array_agg_ordered.rs | 2 +- datafusion/physical-expr/src/aggregate/mod.rs | 12 ------------ .../physical-expr/src/aggregate/nth_value.rs | 2 +- datafusion/physical-plan/src/aggregates/mod.rs | 9 ++++----- 6 files changed, 18 insertions(+), 27 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 055834543c67..d297b5d3574c 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -482,7 +482,7 @@ impl AggregateExpr for FirstValuePhysicalExpr { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::OrderBeneficial + AggregateOrderSensitivity::Beneficial } fn with_requirement_satisfied( @@ -670,7 +670,7 @@ impl AggregateExpr for LastValuePhysicalExpr { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::OrderBeneficial + AggregateOrderSensitivity::Beneficial } fn with_requirement_satisfied( diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 7c57a56720e9..fcb57d826416 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -69,18 +69,22 @@ pub fn create_aggregate_expr( /// Defines how input ordering effects the aggregator #[derive(Debug, Eq, PartialEq, Clone, Copy)] pub enum AggregateOrderSensitivity { - OrderInsensitive, - OrderRequiring, - OrderBeneficial, + Insensitive, + HardRequirement, + Beneficial, } impl AggregateOrderSensitivity { + pub fn is_order_insensitive(&self) -> bool { + self.eq(&AggregateOrderSensitivity::Insensitive) + } + pub fn is_order_beneficial(&self) -> bool { - self.eq(&AggregateOrderSensitivity::OrderBeneficial) + self.eq(&AggregateOrderSensitivity::Beneficial) } pub fn is_order_hard_required(&self) -> bool { - self.eq(&AggregateOrderSensitivity::OrderRequiring) + self.eq(&AggregateOrderSensitivity::HardRequirement) } } @@ -123,7 +127,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// Indicates whether aggregator can produce correct result with any arbitrary ordering or not. fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::OrderInsensitive + AggregateOrderSensitivity::Insensitive } /// Indicates whether requirement of the aggregators is satisfied at the input. diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index b64dd22bf4b9..7b6fdc8cdeaf 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -133,7 +133,7 @@ impl AggregateExpr for OrderSensitiveArrayAgg { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::OrderRequiring + AggregateOrderSensitivity::HardRequirement } fn name(&self) -> &str { diff --git a/datafusion/physical-expr/src/aggregate/mod.rs b/datafusion/physical-expr/src/aggregate/mod.rs index eff008e8f825..0a7cbc0b509c 100644 --- a/datafusion/physical-expr/src/aggregate/mod.rs +++ b/datafusion/physical-expr/src/aggregate/mod.rs @@ -15,10 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::sync::Arc; - -use crate::expressions::{NthValueAgg, OrderSensitiveArrayAgg}; - pub use datafusion_physical_expr_common::aggregate::AggregateExpr; mod hyperloglog; @@ -55,11 +51,3 @@ pub(crate) mod variance; pub mod build_in; pub mod moving_min_max; pub mod utils; - -/// Checks whether the given aggregate expression is order-sensitive. -/// For instance, a `SUM` aggregation doesn't depend on the order of its inputs. -/// However, an `ARRAY_AGG` with `ORDER BY` depends on the input ordering. -pub fn is_order_sensitive(aggr_expr: &Arc) -> bool { - aggr_expr.as_any().is::() - || aggr_expr.as_any().is::() -} diff --git a/datafusion/physical-expr/src/aggregate/nth_value.rs b/datafusion/physical-expr/src/aggregate/nth_value.rs index 846c8bf785bb..ae46aba76152 100644 --- a/datafusion/physical-expr/src/aggregate/nth_value.rs +++ b/datafusion/physical-expr/src/aggregate/nth_value.rs @@ -127,7 +127,7 @@ impl AggregateExpr for NthValueAgg { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { - AggregateOrderSensitivity::OrderRequiring + AggregateOrderSensitivity::HardRequirement } fn name(&self) -> &str { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e1f97b665d55..b0d2da9b2a02 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -39,7 +39,6 @@ use datafusion_common::stats::Precision; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; -use datafusion_physical_expr::aggregate::is_order_sensitive; use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ equivalence::ProjectionMapping, @@ -843,11 +842,11 @@ fn get_aggregate_expr_req( group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { - // If the aggregation function is not order sensitive, or the aggregation - // is performing a "second stage" calculation, or all aggregate function - // requirements are inside the GROUP BY expression, then ignore the ordering + // If the aggregation function is order in-sensitive, or the aggregation + // is performing a "second stage" calculation, then ignore the ordering // requirement. - if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { + if aggr_expr.order_sensitivity().is_order_insensitive() || !agg_mode.is_first_stage() + { return vec![]; } From 98ce018904c6350057f7c5f556e8f404bb5150b0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 17:16:43 +0300 Subject: [PATCH 39/77] Minor changes --- .../src/physical_optimizer/convert_reverse.rs | 53 ++++++++++--------- .../physical-plan/src/aggregates/mod.rs | 5 +- 2 files changed, 30 insertions(+), 28 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_reverse.rs b/datafusion/core/src/physical_optimizer/convert_reverse.rs index 434da1578f7b..82041199240d 100644 --- a/datafusion/core/src/physical_optimizer/convert_reverse.rs +++ b/datafusion/core/src/physical_optimizer/convert_reverse.rs @@ -135,34 +135,35 @@ fn try_convert_reverse_if_better( let reverse_aggr_req = PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - if aggr_expr.order_sensitivity().is_order_hard_required() || aggr_req.is_empty() { - // If ordering for the aggregator is absolute requirement - // or there is no requirement for the aggregator, shouldn't update the aggregator - continue; - } - if eq_properties - .ordering_satisfy_requirement(&concat_slices(prefix_requirement, &aggr_req)) - { - // Existing ordering satisfy the requirement of the aggregator - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to reverse enables more efficient execution - // given the existing ordering: - if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { - *aggr_expr = aggr_expr_rev; + // If ordering for the aggregator is beneficial and there is a requirement for the aggregator, + // try update the aggregator in case there is a more beneficial version with existing ordering + // Otherwise do not update. + if aggr_expr.order_sensitivity().is_order_beneficial() && !aggr_req.is_empty() { + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_req, + )) { + // Existing ordering satisfy the requirement of the aggregator + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to reverse enables more efficient execution + // given the existing ordering: + if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { + *aggr_expr = aggr_expr_rev; + } else { + // If reverse execution is not possible, cannot update current aggregate expression. + continue; + } + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; + // Requirement is not satisfied with existing ordering. } else { - // If reverse execution is not possible, cannot update current aggregate expression. - continue; + // Requirement is not satisfied for the aggregator (Please note that: Aggregator can still work in this case, guaranteed by order sensitive flag being false. + // However, It will be inefficient compared to version where requirement is satisfied). + *aggr_expr = aggr_expr.clone().with_requirement_satisfied(false)?; } - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; - // Requirement is not satisfied with existing ordering. - } else { - // Requirement is not satisfied for the aggregator (Please note that: Aggregator can still work in this case, guaranteed by order sensitive flag being false. - // However, It will be inefficient compared to version where requirement is satisfied). - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(false)?; } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b0d2da9b2a02..6d5a17a0a80e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -842,10 +842,11 @@ fn get_aggregate_expr_req( group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { - // If the aggregation function is order in-sensitive, or the aggregation + // If the aggregation function is ordering requirement is not absolutely necessary, or the aggregation // is performing a "second stage" calculation, then ignore the ordering // requirement. - if aggr_expr.order_sensitivity().is_order_insensitive() || !agg_mode.is_first_stage() + if !aggr_expr.order_sensitivity().is_order_hard_required() + || !agg_mode.is_first_stage() { return vec![]; } From a514db7a49f684cb26a4aaed44d41fa03ed357c4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 17:18:38 +0300 Subject: [PATCH 40/77] Minor changes --- datafusion/physical-expr-common/src/aggregate/mod.rs | 5 ----- 1 file changed, 5 deletions(-) diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index fcb57d826416..709a5d9fba54 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -143,11 +143,6 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { self.name() )); } - println!( - "self.order_sensitivity(): {:?}, aggregator :{:?}", - self.order_sensitivity(), - self.name() - ); unreachable!() } From 99c369bbcfd7ea57b71fdd9f7b4783e57c04b69a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 17:35:22 +0300 Subject: [PATCH 41/77] Minor changes --- .../src/physical_optimizer/convert_reverse.rs | 39 +++++++++---------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_reverse.rs b/datafusion/core/src/physical_optimizer/convert_reverse.rs index 82041199240d..5de9e38a4ea8 100644 --- a/datafusion/core/src/physical_optimizer/convert_reverse.rs +++ b/datafusion/core/src/physical_optimizer/convert_reverse.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::sync::Arc; + +use super::PhysicalOptimizerRule; use datafusion_common::Result; use datafusion_common::{ config::ConfigOptions, @@ -24,14 +27,10 @@ use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalSortRequirement, }; use datafusion_physical_plan::aggregates::concat_slices; +use datafusion_physical_plan::windows::get_ordered_partition_by_indices; use datafusion_physical_plan::{ aggregates::AggregateExec, ExecutionPlan, ExecutionPlanProperties, }; -use std::sync::Arc; - -use datafusion_physical_plan::windows::get_ordered_partition_by_indices; - -use super::PhysicalOptimizerRule; /// The optimizer rule check the ordering requirements of the aggregate expressions. /// And convert between FIRST_VALUE and LAST_VALUE if possible. @@ -57,8 +56,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { plan: Arc, _config: &ConfigOptions, ) -> Result> { - plan.transform_up(&replace_with_reverse_aggregator_when_beneficial) - .data() + plan.transform_up(&update_aggregator_when_beneficial).data() } fn name(&self) -> &str { @@ -70,7 +68,9 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { } } -fn replace_with_reverse_aggregator_when_beneficial( +/// Updates the aggregators with mode `AggregateOrderSensitivity::Beneficial` +/// if existing ordering enables to execute them more efficiently. +fn update_aggregator_when_beneficial( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { @@ -95,7 +95,11 @@ fn replace_with_reverse_aggregator_when_beneficial( }) .collect::>(); - try_convert_reverse_if_better(&requirement, &mut aggr_expr, input_eq_properties)?; + try_convert_aggregate_if_better( + &requirement, + &mut aggr_expr, + input_eq_properties, + )?; let aggr_exec = aggr_exec.with_new_aggr_exprs(aggr_expr); @@ -107,23 +111,18 @@ fn replace_with_reverse_aggregator_when_beneficial( } } -/// Get the common requirement that satisfies all the aggregate expressions. +/// Tries to convert each aggregate expression to a potentially more efficient version. /// /// # Parameters /// -/// - `aggr_exprs`: A slice of `Arc` containing all the -/// aggregate expressions. -/// - `eq_properties`: A reference to an `EquivalenceProperties` instance -/// representing equivalence properties for ordering. +/// * `prefix_requirement` - An array slice representing the ordering requirements preceding the aggregate expressions. +/// * `aggr_exprs` - A mutable slice of `Arc` representing the aggregate expressions to be optimized. +/// * `eq_properties` - A reference to the `EquivalenceProperties` object containing ordering information. /// /// # Returns /// -/// A `LexRequirement` instance, which is the requirement that satisfies all the -/// aggregate requirements. Returns an error in case of conflicting requirements. -/// -/// Similar to the one in datafusion/physical-plan/src/aggregates/mod.rs, but this -/// function care only the possible conversion between FIRST_VALUE and LAST_VALUE -fn try_convert_reverse_if_better( +/// Returns `Ok(())` if the conversion process completes successfully. If an error occurs during the conversion process, an error is returned. +fn try_convert_aggregate_if_better( prefix_requirement: &[PhysicalSortRequirement], aggr_exprs: &mut [Arc], eq_properties: &EquivalenceProperties, From c6fb7a218bdff49e176af87547d2791db2173ae0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 17:43:10 +0300 Subject: [PATCH 42/77] Minor changes --- .../src/physical_optimizer/convert_reverse.rs | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/convert_reverse.rs b/datafusion/core/src/physical_optimizer/convert_reverse.rs index 5de9e38a4ea8..6c5a91269ee4 100644 --- a/datafusion/core/src/physical_optimizer/convert_reverse.rs +++ b/datafusion/core/src/physical_optimizer/convert_reverse.rs @@ -33,14 +33,15 @@ use datafusion_physical_plan::{ }; /// The optimizer rule check the ordering requirements of the aggregate expressions. -/// And convert between FIRST_VALUE and LAST_VALUE if possible. -/// For example, If we have an ascending values and we want LastValue from the descending requirement, -/// it is equivalent to FirstValue with the current ascending ordering. +/// There are 3 kinds of aggregators in terms of ordering requirement +/// - `AggregateOrderSensitivity::Insensitive` +/// - `AggregateOrderSensitivity::HardRequirement` +/// - `AggregateOrderSensitivity::Beneficial` /// -/// The concrete example is that, says we have values c1 with [1, 2, 3], which is an ascending order. -/// If we want LastValue(c1 order by desc), which is the first value of reversed c1 [3, 2, 1], -/// so we can convert the aggregate expression to FirstValue(c1 order by asc), -/// since the current ordering is already satisfied, it saves our time! +/// `AggregateOrderSensitivity::Beneficial` mode have an ordering requirement. However, aggregator can still produce +/// correct result even when ordering requirement is not satisfied (less efficiently). This rule analyzes +/// `AggregateOrderSensitivity::Beneficial` aggregate expressions to see whether their requirement is satisfied or not. +/// Using this information aggregators are updated to either work in efficient mode or less efficient mode. #[derive(Default)] pub struct OptimizeAggregateOrder {} From 36951b95c664f668b3b5d6ff9cd10b38219a485c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 15 Apr 2024 17:59:40 +0300 Subject: [PATCH 43/77] Update comments --- datafusion/core/src/physical_optimizer/mod.rs | 2 +- .../core/src/physical_optimizer/optimizer.rs | 2 +- ...onvert_reverse.rs => update_aggr_exprs.rs} | 22 +++++++++++++++---- .../functions-aggregate/src/first_last.rs | 20 ++++++++++------- .../physical-expr-common/src/aggregate/mod.rs | 19 +++++++++++----- 5 files changed, 45 insertions(+), 20 deletions(-) rename datafusion/core/src/physical_optimizer/{convert_reverse.rs => update_aggr_exprs.rs} (90%) diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index cf8f6860176b..7cc9a0fb75d4 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -24,7 +24,6 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; -mod convert_reverse; pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; @@ -37,6 +36,7 @@ pub mod pruning; pub mod replace_with_order_preserving_variants; mod sort_pushdown; pub mod topk_aggregation; +mod update_aggr_exprs; mod utils; #[cfg(test)] diff --git a/datafusion/core/src/physical_optimizer/optimizer.rs b/datafusion/core/src/physical_optimizer/optimizer.rs index 8a7447bfe932..721dcec99a0d 100644 --- a/datafusion/core/src/physical_optimizer/optimizer.rs +++ b/datafusion/core/src/physical_optimizer/optimizer.rs @@ -19,8 +19,8 @@ use std::sync::Arc; -use super::convert_reverse::OptimizeAggregateOrder; use super::projection_pushdown::ProjectionPushdown; +use super::update_aggr_exprs::OptimizeAggregateOrder; use crate::config::ConfigOptions; use crate::physical_optimizer::aggregate_statistics::AggregateStatistics; use crate::physical_optimizer::coalesce_batches::CoalesceBatches; diff --git a/datafusion/core/src/physical_optimizer/convert_reverse.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs similarity index 90% rename from datafusion/core/src/physical_optimizer/convert_reverse.rs rename to datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 6c5a91269ee4..daebfc4e196a 100644 --- a/datafusion/core/src/physical_optimizer/convert_reverse.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -18,11 +18,11 @@ use std::sync::Arc; use super::PhysicalOptimizerRule; -use datafusion_common::Result; use datafusion_common::{ config::ConfigOptions, tree_node::{Transformed, TransformedResult, TreeNode}, }; +use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalSortRequirement, }; @@ -134,6 +134,11 @@ fn try_convert_aggregate_if_better( let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); let reverse_aggr_req = PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); + let err_fn = || { + plan_datafusion_err!( + "Expects beneficial mode aggregator to implement with_requirement API." + ) + }; // If ordering for the aggregator is beneficial and there is a requirement for the aggregator, // try update the aggregator in case there is a more beneficial version with existing ordering @@ -144,7 +149,10 @@ fn try_convert_aggregate_if_better( &aggr_req, )) { // Existing ordering satisfy the requirement of the aggregator - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; + *aggr_expr = aggr_expr + .clone() + .with_requirement_satisfied(true)? + .ok_or_else(err_fn)?; } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &reverse_aggr_req, @@ -157,12 +165,18 @@ fn try_convert_aggregate_if_better( // If reverse execution is not possible, cannot update current aggregate expression. continue; } - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(true)?; + *aggr_expr = aggr_expr + .clone() + .with_requirement_satisfied(true)? + .ok_or_else(err_fn)?; // Requirement is not satisfied with existing ordering. } else { // Requirement is not satisfied for the aggregator (Please note that: Aggregator can still work in this case, guaranteed by order sensitive flag being false. // However, It will be inefficient compared to version where requirement is satisfied). - *aggr_expr = aggr_expr.clone().with_requirement_satisfied(false)?; + *aggr_expr = aggr_expr + .clone() + .with_requirement_satisfied(false)? + .ok_or_else(err_fn)?; } } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index d297b5d3574c..8da90a25ffe4 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -482,16 +482,18 @@ impl AggregateExpr for FirstValuePhysicalExpr { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { + // Can generate correct result, even if the ordering requirement is not satisfied at the input + // (less efficient compared to ordering satisfied version) AggregateOrderSensitivity::Beneficial } fn with_requirement_satisfied( self: Arc, requirement_satisfied: bool, - ) -> Result> { - let mut res = self.as_ref().clone(); - res.requirement_satisfied = requirement_satisfied; - Ok(Arc::new(res)) + ) -> Result>> { + let mut new_self = self.as_ref().clone(); + new_self.requirement_satisfied = requirement_satisfied; + Ok(Some(Arc::new(new_self))) } fn name(&self) -> &str { @@ -670,16 +672,18 @@ impl AggregateExpr for LastValuePhysicalExpr { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { + // Can generate correct result, even if the ordering requirement is not satisfied at the input + // (less efficient compared to ordering satisfied version) AggregateOrderSensitivity::Beneficial } fn with_requirement_satisfied( self: Arc, requirement_satisfied: bool, - ) -> Result> { - let mut res = self.as_ref().clone(); - res.requirement_satisfied = requirement_satisfied; - Ok(Arc::new(res)) + ) -> Result>> { + let mut new_self = self.as_ref().clone(); + new_self.requirement_satisfied = requirement_satisfied; + Ok(Some(Arc::new(new_self))) } fn name(&self) -> &str { diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 709a5d9fba54..68cd6e959bf2 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -66,14 +66,20 @@ pub fn create_aggregate_expr( })) } -/// Defines how input ordering effects the aggregator -#[derive(Debug, Eq, PartialEq, Clone, Copy)] +/// Represents the sensitivity of an aggregate expression to ordering. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] pub enum AggregateOrderSensitivity { + /// Indicates that the aggregate expression is insensitive to ordering. Ordering at the input + /// is not important for the result of the aggregator Insensitive, + /// Indicates that the aggregate expression has a hard requirement on ordering. Aggregator cannot produce + /// correct result unless its ordering requirement is satisfied. HardRequirement, + /// Indicates that ordering is beneficial for the aggregate expression. Aggregator can produce its result efficiently + /// when its required ordering is satisfied. However, it can still produce correct result (less efficiently) + /// when its required ordering is not satisfied. Beneficial, } - impl AggregateOrderSensitivity { pub fn is_order_insensitive(&self) -> bool { self.eq(&AggregateOrderSensitivity::Insensitive) @@ -126,24 +132,25 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { } /// Indicates whether aggregator can produce correct result with any arbitrary ordering or not. + /// By default we assume aggregate expression is order insensitive. fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } /// Indicates whether requirement of the aggregators is satisfied at the input. - /// If this is not the case some order in-sensitive aggregators can still produce + /// If this is not the case: Aggregators with order sensitivity `AggregateOrderSensitivity::Beneficial` can still produce /// correct result with possibly more work internally. fn with_requirement_satisfied( self: Arc, _requirement_satisfied: bool, - ) -> Result> { + ) -> Result>> { if self.order_bys().is_some() && self.order_sensitivity().is_order_beneficial() { return Err(exec_datafusion_err!( "Should implement with satisfied for aggregator :{:?}", self.name() )); } - unreachable!() + Ok(None) } /// Human readable name such as `"MIN(c2)"`. The default From d00d529662bdb7b42d3a3d8a75253f39158e8881 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 17 Apr 2024 12:51:26 +0300 Subject: [PATCH 44/77] Minor changes --- .../physical_optimizer/update_aggr_exprs.rs | 114 +++++++++--------- .../physical-expr-common/src/aggregate/mod.rs | 9 +- 2 files changed, 68 insertions(+), 55 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index daebfc4e196a..4278ab4bb42e 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -96,9 +96,9 @@ fn update_aggregator_when_beneficial( }) .collect::>(); - try_convert_aggregate_if_better( + aggr_expr = try_convert_aggregate_if_better( + aggr_expr, &requirement, - &mut aggr_expr, input_eq_properties, )?; @@ -116,70 +116,76 @@ fn update_aggregator_when_beneficial( /// /// # Parameters /// +/// * `aggr_exprs` - A evector of `Arc` representing the aggregate expressions to be optimized. /// * `prefix_requirement` - An array slice representing the ordering requirements preceding the aggregate expressions. -/// * `aggr_exprs` - A mutable slice of `Arc` representing the aggregate expressions to be optimized. /// * `eq_properties` - A reference to the `EquivalenceProperties` object containing ordering information. /// /// # Returns /// -/// Returns `Ok(())` if the conversion process completes successfully. If an error occurs during the conversion process, an error is returned. +/// Returns `Ok(converted_aggr_exprs)` if the conversion process completes successfully. +/// If an error occurs during the conversion process, an error is returned. fn try_convert_aggregate_if_better( + aggr_exprs: Vec>, prefix_requirement: &[PhysicalSortRequirement], - aggr_exprs: &mut [Arc], eq_properties: &EquivalenceProperties, -) -> Result<()> { - for aggr_expr in aggr_exprs.iter_mut() { - let aggr_req = aggr_expr.order_bys().unwrap_or(&[]); - let reverse_aggr_req = reverse_order_bys(aggr_req); - let aggr_req = PhysicalSortRequirement::from_sort_exprs(aggr_req); - let reverse_aggr_req = - PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_req); - let err_fn = || { - plan_datafusion_err!( - "Expects beneficial mode aggregator to implement with_requirement API." +) -> Result>> { + aggr_exprs + .into_iter() + .map(|aggr_expr| { + let aggr_sort_exprs = aggr_expr.order_bys().unwrap_or(&[]); + let reverse_aggr_sort_exprs = reverse_order_bys(aggr_sort_exprs); + let aggr_sort_reqs = + PhysicalSortRequirement::from_sort_exprs(aggr_sort_exprs); + let reverse_aggr_req = + PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_sort_exprs); + let err_fn = || { + plan_datafusion_err!( + "Expects an aggregate expression which can benefit from ordered input" ) - }; + }; - // If ordering for the aggregator is beneficial and there is a requirement for the aggregator, - // try update the aggregator in case there is a more beneficial version with existing ordering - // Otherwise do not update. - if aggr_expr.order_sensitivity().is_order_beneficial() && !aggr_req.is_empty() { - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_req, - )) { - // Existing ordering satisfy the requirement of the aggregator - *aggr_expr = aggr_expr - .clone() - .with_requirement_satisfied(true)? - .ok_or_else(err_fn)?; - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { - // Converting to reverse enables more efficient execution - // given the existing ordering: - if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { - *aggr_expr = aggr_expr_rev; + // If ordering for the aggregator is beneficial and there is a requirement for the aggregator, + // try update the aggregator in case there is a more beneficial version with existing ordering + // Otherwise do not update. + if aggr_expr.order_sensitivity().is_order_beneficial() + && !aggr_sort_reqs.is_empty() + { + if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &aggr_sort_reqs, + )) { + // Existing ordering satisfy the requirement of the aggregator + aggr_expr + .with_requirement_satisfied(true)? + .ok_or_else(err_fn) + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { + // Converting to reverse enables more efficient execution + // given the existing ordering: + let updated_aggr_expr = + if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { + aggr_expr_rev + } else { + // If reverse execution is not possible, cannot update current aggregate expression. + aggr_expr + }; + updated_aggr_expr + .with_requirement_satisfied(true)? + .ok_or_else(err_fn) + // Requirement is not satisfied with existing ordering. } else { - // If reverse execution is not possible, cannot update current aggregate expression. - continue; + // Requirement is not satisfied for the aggregator (Please note that: Aggregator can + // still work in this case, guaranteed by order sensitive flag being false. However, + // it will be inefficient compared to version where requirement is satisfied). + aggr_expr + .with_requirement_satisfied(false)? + .ok_or_else(err_fn) } - *aggr_expr = aggr_expr - .clone() - .with_requirement_satisfied(true)? - .ok_or_else(err_fn)?; - // Requirement is not satisfied with existing ordering. } else { - // Requirement is not satisfied for the aggregator (Please note that: Aggregator can still work in this case, guaranteed by order sensitive flag being false. - // However, It will be inefficient compared to version where requirement is satisfied). - *aggr_expr = aggr_expr - .clone() - .with_requirement_satisfied(false)? - .ok_or_else(err_fn)?; + Ok(aggr_expr) } - } - } - - Ok(()) + }) + .collect::>>() } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 68cd6e959bf2..fca05e2c6295 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -137,9 +137,16 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { AggregateOrderSensitivity::Insensitive } - /// Indicates whether requirement of the aggregators is satisfied at the input. + /// Sets the indicator whether requirement of the aggregators is satisfied at the input. /// If this is not the case: Aggregators with order sensitivity `AggregateOrderSensitivity::Beneficial` can still produce /// correct result with possibly more work internally. + /// + /// # Returns + /// + /// Returns `Ok(Some(updated_expr))` if the process completes successfully. + /// If the expression which can benefit does not implement the method, it returns an error. + /// [`AggregateOrderSensitivity::Insensitive`] and [`AggregateOrderSensitivity::HardRequirement`] + /// expressions return Ok(None). fn with_requirement_satisfied( self: Arc, _requirement_satisfied: bool, From 782d1444c1579665ade88e20a4c3ac4aeb12d5d7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 17 Apr 2024 13:20:52 +0300 Subject: [PATCH 45/77] Minor changes --- datafusion/core/src/physical_optimizer/update_aggr_exprs.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 4278ab4bb42e..4b8a545c9be2 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -75,6 +75,9 @@ fn update_aggregator_when_beneficial( plan: Arc, ) -> Result>> { if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // Final stage implementations doesn't rely on ordering (Those ordering fields may be pruned out by + // first stage aggregates. Hence, necessary information for proper merge is added during the first stage to the state field. + // Final stage only uses the state field information). if !aggr_exec.mode().is_first_stage() { return Ok(Transformed::no(plan)); } From 56c7629993dd0a681f3a5a3beae6fd7dcc522eab Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 24 Apr 2024 15:12:50 +0300 Subject: [PATCH 46/77] Review Part 1 --- .../physical_optimizer/update_aggr_exprs.rs | 191 +++++++++--------- .../functions-aggregate/src/first_last.rs | 8 +- 2 files changed, 97 insertions(+), 102 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 4b8a545c9be2..1e57a2101921 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -15,13 +15,15 @@ // specific language governing permissions and limitations // under the License. +//! An optimizer rule that checks ordering requirements of aggregate expressions +//! and modifies the expressions to work more efficiently if possible. + use std::sync::Arc; use super::PhysicalOptimizerRule; -use datafusion_common::{ - config::ConfigOptions, - tree_node::{Transformed, TransformedResult, TreeNode}, -}; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{plan_datafusion_err, Result}; use datafusion_physical_expr::{ reverse_order_bys, AggregateExpr, EquivalenceProperties, PhysicalSortRequirement, @@ -32,16 +34,20 @@ use datafusion_physical_plan::{ aggregates::AggregateExec, ExecutionPlan, ExecutionPlanProperties, }; -/// The optimizer rule check the ordering requirements of the aggregate expressions. -/// There are 3 kinds of aggregators in terms of ordering requirement -/// - `AggregateOrderSensitivity::Insensitive` -/// - `AggregateOrderSensitivity::HardRequirement` -/// - `AggregateOrderSensitivity::Beneficial` +/// This optimizer rule checks ordering requirements of aggregate expressions. /// -/// `AggregateOrderSensitivity::Beneficial` mode have an ordering requirement. However, aggregator can still produce -/// correct result even when ordering requirement is not satisfied (less efficiently). This rule analyzes -/// `AggregateOrderSensitivity::Beneficial` aggregate expressions to see whether their requirement is satisfied or not. -/// Using this information aggregators are updated to either work in efficient mode or less efficient mode. +/// There are 3 kinds of aggregators in terms of ordering requirements: +/// - `AggregateOrderSensitivity::Insensitive`, meaning that ordering is not +/// important. +/// - `AggregateOrderSensitivity::HardRequirement`, meaning that the aggregator +/// requires a specific ordering. +/// - `AggregateOrderSensitivity::Beneficial`, meaning that the aggregator can +/// handle unordered input, but can run more efficiently if its input conforms +/// to a specific ordering. +/// +/// This rule analyzes aggregate expressions of type `Beneficial` to see whether +/// their input ordering requirements are satisfied. If this is the case, the +/// aggregators are modified to run in a more efficient mode. #[derive(Default)] pub struct OptimizeAggregateOrder {} @@ -57,7 +63,44 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { plan: Arc, _config: &ConfigOptions, ) -> Result> { - plan.transform_up(&update_aggregator_when_beneficial).data() + plan.transform_up(|plan| { + if let Some(aggr_exec) = plan.as_any().downcast_ref::() { + // Final stage implementations do not rely on ordering -- those + // ordering fields may be pruned out by first stage aggregates. + // Hence, necessary information for proper merge is added during + // the first stage to the state field, which the final stage uses. + if !aggr_exec.mode().is_first_stage() { + return Ok(Transformed::no(plan)); + } + let input = aggr_exec.input(); + let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); + + let groupby_exprs = aggr_exec.group_by().input_exprs(); + // If the existing ordering satisfies a prefix of the GROUP BY + // expressions, prefix requirements with this section. In this + // case, aggregation will work more efficiently. + let indices = get_ordered_partition_by_indices(&groupby_exprs, input); + let requirement = indices + .iter() + .map(|&idx| { + PhysicalSortRequirement::new(groupby_exprs[idx].clone(), None) + }) + .collect::>(); + + aggr_expr = try_convert_aggregate_if_better( + aggr_expr, + &requirement, + input.equivalence_properties(), + )?; + + let aggr_exec = aggr_exec.with_new_aggr_exprs(aggr_expr); + + Ok(Transformed::yes(Arc::new(aggr_exec) as _)) + } else { + Ok(Transformed::no(plan)) + } + }) + .data() } fn name(&self) -> &str { @@ -69,69 +112,33 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { } } -/// Updates the aggregators with mode `AggregateOrderSensitivity::Beneficial` -/// if existing ordering enables to execute them more efficiently. -fn update_aggregator_when_beneficial( - plan: Arc, -) -> Result>> { - if let Some(aggr_exec) = plan.as_any().downcast_ref::() { - // Final stage implementations doesn't rely on ordering (Those ordering fields may be pruned out by - // first stage aggregates. Hence, necessary information for proper merge is added during the first stage to the state field. - // Final stage only uses the state field information). - if !aggr_exec.mode().is_first_stage() { - return Ok(Transformed::no(plan)); - } - let input = aggr_exec.input(); - let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); - let group_by = aggr_exec.group_by(); - - let input_eq_properties = input.equivalence_properties(); - let groupby_exprs = group_by.input_exprs(); - // If existing ordering satisfies a prefix of the GROUP BY expressions, - // prefix requirements with this section. In this case, aggregation will - // work more efficiently. - let indices = get_ordered_partition_by_indices(&groupby_exprs, input); - let requirement = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - - aggr_expr = try_convert_aggregate_if_better( - aggr_expr, - &requirement, - input_eq_properties, - )?; - - let aggr_exec = aggr_exec.with_new_aggr_exprs(aggr_expr); - - Ok(Transformed::yes( - Arc::new(aggr_exec) as Arc - )) - } else { - Ok(Transformed::no(plan)) - } -} - -/// Tries to convert each aggregate expression to a potentially more efficient version. +/// Tries to convert each aggregate expression to a potentially more efficient +/// version. /// /// # Parameters /// -/// * `aggr_exprs` - A evector of `Arc` representing the aggregate expressions to be optimized. -/// * `prefix_requirement` - An array slice representing the ordering requirements preceding the aggregate expressions. -/// * `eq_properties` - A reference to the `EquivalenceProperties` object containing ordering information. +/// * `aggr_exprs` - A vector of `Arc` representing the +/// aggregate expressions to be optimized. +/// * `prefix_requirement` - An array slice representing the ordering +/// requirements preceding the aggregate expressions. +/// * `eq_properties` - A reference to the `EquivalenceProperties` object +/// containing ordering information. /// /// # Returns /// -/// Returns `Ok(converted_aggr_exprs)` if the conversion process completes successfully. -/// If an error occurs during the conversion process, an error is returned. +/// Returns `Ok(converted_aggr_exprs)` if the conversion process completes +/// successfully. Any errors occuring during the conversion process are +/// passed through. fn try_convert_aggregate_if_better( aggr_exprs: Vec>, prefix_requirement: &[PhysicalSortRequirement], eq_properties: &EquivalenceProperties, ) -> Result>> { + let err_fn = || { + plan_datafusion_err!( + "Expects an aggregate expression that can benefit from input ordering" + ) + }; aggr_exprs .into_iter() .map(|aggr_expr| { @@ -141,47 +148,33 @@ fn try_convert_aggregate_if_better( PhysicalSortRequirement::from_sort_exprs(aggr_sort_exprs); let reverse_aggr_req = PhysicalSortRequirement::from_sort_exprs(&reverse_aggr_sort_exprs); - let err_fn = || { - plan_datafusion_err!( - "Expects an aggregate expression which can benefit from ordered input" - ) - }; - // If ordering for the aggregator is beneficial and there is a requirement for the aggregator, - // try update the aggregator in case there is a more beneficial version with existing ordering - // Otherwise do not update. + // If the aggregate expression benefits from input ordering, and + // there is an actual ordering enabling this, try to update the + // aggregate expression to benefit from the existing ordering. + // Otherwise, leave it as is. if aggr_expr.order_sensitivity().is_order_beneficial() && !aggr_sort_reqs.is_empty() { - if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &aggr_sort_reqs, - )) { - // Existing ordering satisfy the requirement of the aggregator - aggr_expr + let reqs = concat_slices(prefix_requirement, &aggr_sort_reqs); + if eq_properties.ordering_satisfy_requirement(&reqs) { + // Existing ordering satisfies the aggregator requirements: + return aggr_expr .with_requirement_satisfied(true)? - .ok_or_else(err_fn) - } else if eq_properties.ordering_satisfy_requirement(&concat_slices( - prefix_requirement, - &reverse_aggr_req, - )) { + .ok_or_else(err_fn); + } + let reqs = concat_slices(prefix_requirement, &reverse_aggr_req); + if eq_properties.ordering_satisfy_requirement(&reqs) { // Converting to reverse enables more efficient execution - // given the existing ordering: - let updated_aggr_expr = - if let Some(aggr_expr_rev) = aggr_expr.reverse_expr() { - aggr_expr_rev - } else { - // If reverse execution is not possible, cannot update current aggregate expression. - aggr_expr - }; - updated_aggr_expr + // given the existing ordering (if possible): + aggr_expr + .reverse_expr() + .unwrap_or(aggr_expr) .with_requirement_satisfied(true)? .ok_or_else(err_fn) - // Requirement is not satisfied with existing ordering. } else { - // Requirement is not satisfied for the aggregator (Please note that: Aggregator can - // still work in this case, guaranteed by order sensitive flag being false. However, - // it will be inefficient compared to version where requirement is satisfied). + // There is no beneficial ordering present -- aggregation + // will still work albeit in a less efficient mode. aggr_expr .with_requirement_satisfied(false)? .ok_or_else(err_fn) @@ -190,5 +183,5 @@ fn try_convert_aggregate_if_better( Ok(aggr_expr) } }) - .collect::>>() + .collect() } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 7559dc055fcb..cba8d8f3b193 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -17,6 +17,10 @@ //! Defines the FIRST_VALUE/LAST_VALUE aggregations. +use std::any::Any; +use std::fmt::Debug; +use std::sync::Arc; + use arrow::array::{ArrayRef, AsArray, BooleanArray}; use arrow::compute::{self, lexsort_to_indices, SortColumn, SortOptions}; use arrow::datatypes::{DataType, Field}; @@ -41,10 +45,8 @@ use datafusion_physical_expr_common::expressions; use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_expr_common::utils::reverse_order_bys; + use sqlparser::ast::NullTreatment; -use std::any::Any; -use std::fmt::Debug; -use std::sync::Arc; make_udaf_function!( FirstValue, From e1883d96aae4b56b8d792352446bd1de39c4b97f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 09:11:07 +0300 Subject: [PATCH 47/77] TMP --- datafusion/core/src/prelude.rs | 1 + datafusion/expr/src/aggregate_function.rs | 16 +- .../expr/src/type_coercion/aggregates.rs | 4 +- .../functions-aggregate/src/first_last.rs | 818 ++++++++++-------- datafusion/functions-aggregate/src/lib.rs | 6 +- datafusion/optimizer/Cargo.toml | 1 + .../src/replace_distinct_aggregate.rs | 15 +- .../physical-expr/src/aggregate/build_in.rs | 21 - .../src/equivalence/projection.rs | 10 +- .../physical-expr/src/expressions/mod.rs | 4 +- 10 files changed, 486 insertions(+), 410 deletions(-) diff --git a/datafusion/core/src/prelude.rs b/datafusion/core/src/prelude.rs index d82a5a2cc1a1..0d8d06f49bc3 100644 --- a/datafusion/core/src/prelude.rs +++ b/datafusion/core/src/prelude.rs @@ -39,6 +39,7 @@ pub use datafusion_expr::{ Expr, }; pub use datafusion_functions::expr_fn::*; +pub use datafusion_functions_aggregate::expr_fn::*; #[cfg(feature = "array_expressions")] pub use datafusion_functions_array::expr_fn::*; diff --git a/datafusion/expr/src/aggregate_function.rs b/datafusion/expr/src/aggregate_function.rs index 3dc9c3a01c15..eef83ac78e48 100644 --- a/datafusion/expr/src/aggregate_function.rs +++ b/datafusion/expr/src/aggregate_function.rs @@ -49,10 +49,6 @@ pub enum AggregateFunction { ApproxDistinct, /// Aggregation into an array ArrayAgg, - /// First value in a group according to some ordering - FirstValue, - /// Last value in a group according to some ordering - LastValue, /// N'th value in a group according to some ordering NthValue, /// Variance (Sample) @@ -121,8 +117,6 @@ impl AggregateFunction { Median => "MEDIAN", ApproxDistinct => "APPROX_DISTINCT", ArrayAgg => "ARRAY_AGG", - FirstValue => "FIRST_VALUE", - LastValue => "LAST_VALUE", NthValue => "NTH_VALUE", Variance => "VAR", VariancePop => "VAR_POP", @@ -178,8 +172,6 @@ impl FromStr for AggregateFunction { "min" => AggregateFunction::Min, "sum" => AggregateFunction::Sum, "array_agg" => AggregateFunction::ArrayAgg, - "first_value" => AggregateFunction::FirstValue, - "last_value" => AggregateFunction::LastValue, "nth_value" => AggregateFunction::NthValue, "string_agg" => AggregateFunction::StringAgg, // statistical @@ -294,9 +286,7 @@ impl AggregateFunction { Ok(coerced_data_types[0].clone()) } AggregateFunction::Grouping => Ok(DataType::Int32), - AggregateFunction::FirstValue - | AggregateFunction::LastValue - | AggregateFunction::NthValue => Ok(coerced_data_types[0].clone()), + AggregateFunction::NthValue => Ok(coerced_data_types[0].clone()), AggregateFunction::StringAgg => Ok(DataType::LargeUtf8), } } @@ -351,9 +341,7 @@ impl AggregateFunction { | AggregateFunction::Stddev | AggregateFunction::StddevPop | AggregateFunction::Median - | AggregateFunction::ApproxMedian - | AggregateFunction::FirstValue - | AggregateFunction::LastValue => { + | AggregateFunction::ApproxMedian => { Signature::uniform(1, NUMERICS.to_vec(), Volatility::Immutable) } AggregateFunction::NthValue => Signature::any(2, Volatility::Immutable), diff --git a/datafusion/expr/src/type_coercion/aggregates.rs b/datafusion/expr/src/type_coercion/aggregates.rs index 5ffdc8f94753..c8bada461d62 100644 --- a/datafusion/expr/src/type_coercion/aggregates.rs +++ b/datafusion/expr/src/type_coercion/aggregates.rs @@ -293,9 +293,7 @@ pub fn coerce_types( } Ok(input_types.to_vec()) } - AggregateFunction::Median - | AggregateFunction::FirstValue - | AggregateFunction::LastValue => Ok(input_types.to_vec()), + AggregateFunction::Median => Ok(input_types.to_vec()), AggregateFunction::NthValue => Ok(input_types.to_vec()), AggregateFunction::Grouping => Ok(vec![input_types[0].clone()]), AggregateFunction::StringAgg => { diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 7559dc055fcb..a1b85abaae23 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -31,20 +31,12 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, Signature, TypeSignature, Volatility, }; -use datafusion_physical_expr_common::aggregate::utils::{ - down_cast_any_ref, get_sort_options, ordering_fields, -}; -use datafusion_physical_expr_common::aggregate::{ - AggregateExpr, AggregateOrderSensitivity, -}; +use datafusion_physical_expr_common::aggregate::utils::get_sort_options; use datafusion_physical_expr_common::expressions; -use datafusion_physical_expr_common::physical_expr::PhysicalExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_expr_common::utils::reverse_order_bys; use sqlparser::ast::NullTreatment; use std::any::Any; use std::fmt::Debug; -use std::sync::Arc; make_udaf_function!( FirstValue, @@ -77,7 +69,7 @@ impl Default for FirstValue { impl FirstValue { pub fn new() -> Self { Self { - aliases: vec![String::from("FIRST_VALUE")], + aliases: vec![String::from("FIRST_VALUE"), String::from("first_value")], signature: Signature::one_of( vec![ // TODO: we can introduce more strict signature that only numeric of array types are allowed @@ -345,385 +337,509 @@ impl Accumulator for FirstValueAccumulator { } } -/// TO BE DEPRECATED: Builtin FIRST_VALUE physical aggregate expression will be replaced by udf in the future -#[derive(Debug, Clone)] -pub struct FirstValuePhysicalExpr { - name: String, - input_data_type: DataType, - order_by_data_types: Vec, - expr: Arc, - ordering_req: LexOrdering, - requirement_satisfied: bool, - ignore_nulls: bool, - state_fields: Vec, -} - -impl FirstValuePhysicalExpr { - /// Creates a new FIRST_VALUE aggregation function. - pub fn new( - expr: Arc, - name: impl Into, - input_data_type: DataType, - ordering_req: LexOrdering, - order_by_data_types: Vec, - state_fields: Vec, - ) -> Self { - let requirement_satisfied = ordering_req.is_empty(); - Self { - name: name.into(), - input_data_type, - order_by_data_types, - expr, - ordering_req, - requirement_satisfied, - ignore_nulls: false, - state_fields, - } - } - - pub fn with_ignore_nulls(mut self, ignore_nulls: bool) -> Self { - self.ignore_nulls = ignore_nulls; - self - } - - /// Returns the name of the aggregate expression. - pub fn name(&self) -> &str { - &self.name - } - - /// Returns the input data type of the aggregate expression. - pub fn input_data_type(&self) -> &DataType { - &self.input_data_type - } - - /// Returns the data types of the order-by columns. - pub fn order_by_data_types(&self) -> &Vec { - &self.order_by_data_types - } - - /// Returns the expression associated with the aggregate function. - pub fn expr(&self) -> &Arc { - &self.expr - } - - /// Returns the lexical ordering requirements of the aggregate expression. - pub fn ordering_req(&self) -> &LexOrdering { - &self.ordering_req - } - - pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; - self - } +// /// TO BE DEPRECATED: Builtin FIRST_VALUE physical aggregate expression will be replaced by udf in the future +// #[derive(Debug, Clone)] +// pub struct FirstValuePhysicalExpr { +// name: String, +// input_data_type: DataType, +// order_by_data_types: Vec, +// expr: Arc, +// ordering_req: LexOrdering, +// requirement_satisfied: bool, +// ignore_nulls: bool, +// state_fields: Vec, +// } +// +// impl FirstValuePhysicalExpr { +// /// Creates a new FIRST_VALUE aggregation function. +// pub fn new( +// expr: Arc, +// name: impl Into, +// input_data_type: DataType, +// ordering_req: LexOrdering, +// order_by_data_types: Vec, +// state_fields: Vec, +// ) -> Self { +// let requirement_satisfied = ordering_req.is_empty(); +// Self { +// name: name.into(), +// input_data_type, +// order_by_data_types, +// expr, +// ordering_req, +// requirement_satisfied, +// ignore_nulls: false, +// state_fields, +// } +// } +// +// pub fn with_ignore_nulls(mut self, ignore_nulls: bool) -> Self { +// self.ignore_nulls = ignore_nulls; +// self +// } +// +// /// Returns the name of the aggregate expression. +// pub fn name(&self) -> &str { +// &self.name +// } +// +// /// Returns the input data type of the aggregate expression. +// pub fn input_data_type(&self) -> &DataType { +// &self.input_data_type +// } +// +// /// Returns the data types of the order-by columns. +// pub fn order_by_data_types(&self) -> &Vec { +// &self.order_by_data_types +// } +// +// /// Returns the expression associated with the aggregate function. +// pub fn expr(&self) -> &Arc { +// &self.expr +// } +// +// /// Returns the lexical ordering requirements of the aggregate expression. +// pub fn ordering_req(&self) -> &LexOrdering { +// &self.ordering_req +// } +// +// pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { +// self.requirement_satisfied = requirement_satisfied; +// self +// } +// +// pub fn convert_to_last(self) -> LastValuePhysicalExpr { +// let mut name = format!("LAST{}", &self.name[5..]); +// replace_order_by_clause(&mut name); +// +// let FirstValuePhysicalExpr { +// expr, +// input_data_type, +// ordering_req, +// order_by_data_types, +// .. +// } = self; +// LastValuePhysicalExpr::new( +// expr, +// name, +// input_data_type, +// reverse_order_bys(&ordering_req), +// order_by_data_types, +// ) +// } +// } +// +// impl AggregateExpr for FirstValuePhysicalExpr { +// /// 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.input_data_type.clone(), true)) +// } +// +// fn create_accumulator(&self) -> Result> { +// FirstValueAccumulator::try_new( +// &self.input_data_type, +// &self.order_by_data_types, +// self.ordering_req.clone(), +// self.ignore_nulls, +// ) +// .map(|acc| { +// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ +// }) +// } +// +// fn state_fields(&self) -> Result> { +// if !self.state_fields.is_empty() { +// return Ok(self.state_fields.clone()); +// } +// +// let mut fields = vec![Field::new( +// format_state_name(&self.name, "first_value"), +// self.input_data_type.clone(), +// true, +// )]; +// fields.extend(ordering_fields( +// &self.ordering_req, +// &self.order_by_data_types, +// )); +// fields.push(Field::new( +// format_state_name(&self.name, "is_set"), +// DataType::Boolean, +// true, +// )); +// Ok(fields) +// } +// +// fn expressions(&self) -> Vec> { +// vec![self.expr.clone()] +// } +// +// fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { +// (!self.ordering_req.is_empty()).then_some(&self.ordering_req) +// } +// +// fn order_sensitivity(&self) -> AggregateOrderSensitivity { +// // Can generate correct result, even if the ordering requirement is not satisfied at the input +// // (less efficient compared to ordering satisfied version) +// AggregateOrderSensitivity::Beneficial +// } +// +// fn with_requirement_satisfied( +// self: Arc, +// requirement_satisfied: bool, +// ) -> Result>> { +// let mut new_self = self.as_ref().clone(); +// new_self.requirement_satisfied = requirement_satisfied; +// Ok(Some(Arc::new(new_self))) +// } +// +// fn name(&self) -> &str { +// &self.name +// } +// +// fn reverse_expr(&self) -> Option> { +// Some(Arc::new(self.clone().convert_to_last())) +// } +// +// fn create_sliding_accumulator(&self) -> Result> { +// FirstValueAccumulator::try_new( +// &self.input_data_type, +// &self.order_by_data_types, +// self.ordering_req.clone(), +// self.ignore_nulls, +// ) +// .map(|acc| { +// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ +// }) +// } +// } +// +// impl PartialEq for FirstValuePhysicalExpr { +// fn eq(&self, other: &dyn Any) -> bool { +// down_cast_any_ref(other) +// .downcast_ref::() +// .map(|x| { +// self.name == x.name +// && self.input_data_type == x.input_data_type +// && self.order_by_data_types == x.order_by_data_types +// && self.expr.eq(&x.expr) +// }) +// .unwrap_or(false) +// } +// } +// +// /// TO BE DEPRECATED: Builtin LAST_VALUE physical aggregate expression will be replaced by udf in the future +// #[derive(Debug, Clone)] +// pub struct LastValuePhysicalExpr { +// name: String, +// input_data_type: DataType, +// order_by_data_types: Vec, +// expr: Arc, +// ordering_req: LexOrdering, +// requirement_satisfied: bool, +// ignore_nulls: bool, +// } +// +// impl LastValuePhysicalExpr { +// /// Creates a new LAST_VALUE aggregation function. +// pub fn new( +// expr: Arc, +// name: impl Into, +// input_data_type: DataType, +// ordering_req: LexOrdering, +// order_by_data_types: Vec, +// ) -> Self { +// let requirement_satisfied = ordering_req.is_empty(); +// Self { +// name: name.into(), +// input_data_type, +// order_by_data_types, +// expr, +// ordering_req, +// requirement_satisfied, +// ignore_nulls: false, +// } +// } +// +// pub fn with_ignore_nulls(mut self, ignore_nulls: bool) -> Self { +// self.ignore_nulls = ignore_nulls; +// self +// } +// +// /// Returns the name of the aggregate expression. +// pub fn name(&self) -> &str { +// &self.name +// } +// +// /// Returns the input data type of the aggregate expression. +// pub fn input_data_type(&self) -> &DataType { +// &self.input_data_type +// } +// +// /// Returns the data types of the order-by columns. +// pub fn order_by_data_types(&self) -> &Vec { +// &self.order_by_data_types +// } +// +// /// Returns the expression associated with the aggregate function. +// pub fn expr(&self) -> &Arc { +// &self.expr +// } +// +// /// Returns the lexical ordering requirements of the aggregate expression. +// pub fn ordering_req(&self) -> &LexOrdering { +// &self.ordering_req +// } +// +// pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { +// self.requirement_satisfied = requirement_satisfied; +// self +// } +// +// pub fn convert_to_first(self) -> FirstValuePhysicalExpr { +// let mut name = format!("FIRST{}", &self.name[4..]); +// replace_order_by_clause(&mut name); +// +// let LastValuePhysicalExpr { +// expr, +// input_data_type, +// ordering_req, +// order_by_data_types, +// .. +// } = self; +// FirstValuePhysicalExpr::new( +// expr, +// name, +// input_data_type, +// reverse_order_bys(&ordering_req), +// order_by_data_types, +// vec![], +// ) +// } +// } +// +// impl AggregateExpr for LastValuePhysicalExpr { +// /// 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.input_data_type.clone(), true)) +// } +// +// fn create_accumulator(&self) -> Result> { +// LastValueAccumulator::try_new( +// &self.input_data_type, +// &self.order_by_data_types, +// self.ordering_req.clone(), +// self.ignore_nulls, +// ) +// .map(|acc| { +// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ +// }) +// } +// +// fn state_fields(&self) -> Result> { +// let mut fields = vec![Field::new( +// format_state_name(&self.name, "last_value"), +// self.input_data_type.clone(), +// true, +// )]; +// fields.extend(ordering_fields( +// &self.ordering_req, +// &self.order_by_data_types, +// )); +// fields.push(Field::new( +// format_state_name(&self.name, "is_set"), +// DataType::Boolean, +// true, +// )); +// Ok(fields) +// } +// +// fn expressions(&self) -> Vec> { +// vec![self.expr.clone()] +// } +// +// fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { +// (!self.ordering_req.is_empty()).then_some(&self.ordering_req) +// } +// +// fn order_sensitivity(&self) -> AggregateOrderSensitivity { +// // Can generate correct result, even if the ordering requirement is not satisfied at the input +// // (less efficient compared to ordering satisfied version) +// AggregateOrderSensitivity::Beneficial +// } +// +// fn with_requirement_satisfied( +// self: Arc, +// requirement_satisfied: bool, +// ) -> Result>> { +// let mut new_self = self.as_ref().clone(); +// new_self.requirement_satisfied = requirement_satisfied; +// Ok(Some(Arc::new(new_self))) +// } +// +// fn name(&self) -> &str { +// &self.name +// } +// +// fn reverse_expr(&self) -> Option> { +// Some(Arc::new(self.clone().convert_to_first())) +// } +// +// fn create_sliding_accumulator(&self) -> Result> { +// LastValueAccumulator::try_new( +// &self.input_data_type, +// &self.order_by_data_types, +// self.ordering_req.clone(), +// self.ignore_nulls, +// ) +// .map(|acc| { +// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ +// }) +// } +// } +// +// impl PartialEq for LastValuePhysicalExpr { +// fn eq(&self, other: &dyn Any) -> bool { +// down_cast_any_ref(other) +// .downcast_ref::() +// .map(|x| { +// self.name == x.name +// && self.input_data_type == x.input_data_type +// && self.order_by_data_types == x.order_by_data_types +// && self.expr.eq(&x.expr) +// }) +// .unwrap_or(false) +// } +// } - pub fn convert_to_last(self) -> LastValuePhysicalExpr { - let mut name = format!("LAST{}", &self.name[5..]); - replace_order_by_clause(&mut name); +make_udaf_function!( + LastValue, + last_value, + "Returns the last value in a group of values.", + last_value_udaf +); - let FirstValuePhysicalExpr { - expr, - input_data_type, - ordering_req, - order_by_data_types, - .. - } = self; - LastValuePhysicalExpr::new( - expr, - name, - input_data_type, - reverse_order_bys(&ordering_req), - order_by_data_types, - ) - } +pub struct LastValue { + signature: Signature, + aliases: Vec, } -impl AggregateExpr for FirstValuePhysicalExpr { - /// 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.input_data_type.clone(), true)) - } - - fn create_accumulator(&self) -> Result> { - FirstValueAccumulator::try_new( - &self.input_data_type, - &self.order_by_data_types, - self.ordering_req.clone(), - self.ignore_nulls, - ) - .map(|acc| { - Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ - }) - } - - fn state_fields(&self) -> Result> { - if !self.state_fields.is_empty() { - return Ok(self.state_fields.clone()); - } - - let mut fields = vec![Field::new( - format_state_name(&self.name, "first_value"), - self.input_data_type.clone(), - true, - )]; - fields.extend(ordering_fields( - &self.ordering_req, - &self.order_by_data_types, - )); - fields.push(Field::new( - format_state_name(&self.name, "is_set"), - DataType::Boolean, - true, - )); - Ok(fields) - } - - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - (!self.ordering_req.is_empty()).then_some(&self.ordering_req) - } - - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - // Can generate correct result, even if the ordering requirement is not satisfied at the input - // (less efficient compared to ordering satisfied version) - AggregateOrderSensitivity::Beneficial - } - - fn with_requirement_satisfied( - self: Arc, - requirement_satisfied: bool, - ) -> Result>> { - let mut new_self = self.as_ref().clone(); - new_self.requirement_satisfied = requirement_satisfied; - Ok(Some(Arc::new(new_self))) - } - - fn name(&self) -> &str { - &self.name - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone().convert_to_last())) - } - - fn create_sliding_accumulator(&self) -> Result> { - FirstValueAccumulator::try_new( - &self.input_data_type, - &self.order_by_data_types, - self.ordering_req.clone(), - self.ignore_nulls, - ) - .map(|acc| { - Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ - }) +impl Debug for LastValue { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + f.debug_struct("LastValue") + .field("name", &self.name()) + .field("signature", &self.signature) + .field("accumulator", &"") + .finish() } } -impl PartialEq for FirstValuePhysicalExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.input_data_type == x.input_data_type - && self.order_by_data_types == x.order_by_data_types - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) +impl Default for LastValue { + fn default() -> Self { + Self::new() } } -/// TO BE DEPRECATED: Builtin LAST_VALUE physical aggregate expression will be replaced by udf in the future -#[derive(Debug, Clone)] -pub struct LastValuePhysicalExpr { - name: String, - input_data_type: DataType, - order_by_data_types: Vec, - expr: Arc, - ordering_req: LexOrdering, - requirement_satisfied: bool, - ignore_nulls: bool, -} - -impl LastValuePhysicalExpr { - /// Creates a new LAST_VALUE aggregation function. - pub fn new( - expr: Arc, - name: impl Into, - input_data_type: DataType, - ordering_req: LexOrdering, - order_by_data_types: Vec, - ) -> Self { - let requirement_satisfied = ordering_req.is_empty(); +impl LastValue { + pub fn new() -> Self { Self { - name: name.into(), - input_data_type, - order_by_data_types, - expr, - ordering_req, - requirement_satisfied, - ignore_nulls: false, + aliases: vec![String::from("LAST_VALUE"), String::from("last_value")], + signature: Signature::one_of( + vec![ + // TODO: we can introduce more strict signature that only numeric of array types are allowed + TypeSignature::ArraySignature(ArrayFunctionSignature::Array), + TypeSignature::Uniform(1, NUMERICS.to_vec()), + ], + Volatility::Immutable, + ), } } +} - pub fn with_ignore_nulls(mut self, ignore_nulls: bool) -> Self { - self.ignore_nulls = ignore_nulls; +impl AggregateUDFImpl for LastValue { + fn as_any(&self) -> &dyn Any { self } - /// Returns the name of the aggregate expression. - pub fn name(&self) -> &str { - &self.name - } - - /// Returns the input data type of the aggregate expression. - pub fn input_data_type(&self) -> &DataType { - &self.input_data_type - } - - /// Returns the data types of the order-by columns. - pub fn order_by_data_types(&self) -> &Vec { - &self.order_by_data_types + fn name(&self) -> &str { + "LAST_VALUE" } - /// Returns the expression associated with the aggregate function. - pub fn expr(&self) -> &Arc { - &self.expr + fn signature(&self) -> &Signature { + &self.signature } - /// Returns the lexical ordering requirements of the aggregate expression. - pub fn ordering_req(&self) -> &LexOrdering { - &self.ordering_req + fn return_type(&self, arg_types: &[DataType]) -> Result { + Ok(arg_types[0].clone()) } - pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; - self - } + fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { + let mut all_sort_orders = vec![]; - pub fn convert_to_first(self) -> FirstValuePhysicalExpr { - let mut name = format!("FIRST{}", &self.name[4..]); - replace_order_by_clause(&mut name); + // Construct PhysicalSortExpr objects from Expr objects: + let mut sort_exprs = vec![]; + for expr in acc_args.sort_exprs { + if let Expr::Sort(sort) = expr { + if let Expr::Column(col) = sort.expr.as_ref() { + let name = &col.name; + let e = expressions::column::col(name, acc_args.schema)?; + sort_exprs.push(PhysicalSortExpr { + expr: e, + options: SortOptions { + descending: !sort.asc, + nulls_first: sort.nulls_first, + }, + }); + } + } + } + if !sort_exprs.is_empty() { + all_sort_orders.extend(sort_exprs); + } - let LastValuePhysicalExpr { - expr, - input_data_type, - ordering_req, - order_by_data_types, - .. - } = self; - FirstValuePhysicalExpr::new( - expr, - name, - input_data_type, - reverse_order_bys(&ordering_req), - order_by_data_types, - vec![], - ) - } -} + let ordering_req = all_sort_orders; -impl AggregateExpr for LastValuePhysicalExpr { - /// Return a reference to Any that can be used for downcasting - fn as_any(&self) -> &dyn Any { - self - } + let ordering_dtypes = ordering_req + .iter() + .map(|e| e.expr.data_type(acc_args.schema)) + .collect::>>()?; - fn field(&self) -> Result { - Ok(Field::new(&self.name, self.input_data_type.clone(), true)) - } + let requirement_satisfied = ordering_req.is_empty(); - fn create_accumulator(&self) -> Result> { LastValueAccumulator::try_new( - &self.input_data_type, - &self.order_by_data_types, - self.ordering_req.clone(), - self.ignore_nulls, + acc_args.data_type, + &ordering_dtypes, + ordering_req, + acc_args.ignore_nulls, ) - .map(|acc| { - Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ - }) + .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) } - fn state_fields(&self) -> Result> { + fn state_fields( + &self, + name: &str, + value_type: DataType, + ordering_fields: Vec, + ) -> Result> { let mut fields = vec![Field::new( - format_state_name(&self.name, "last_value"), - self.input_data_type.clone(), + format_state_name(name, "last_value"), + value_type, true, )]; - fields.extend(ordering_fields( - &self.ordering_req, - &self.order_by_data_types, - )); - fields.push(Field::new( - format_state_name(&self.name, "is_set"), - DataType::Boolean, - true, - )); + fields.extend(ordering_fields); + fields.push(Field::new("is_set", DataType::Boolean, true)); Ok(fields) } - fn expressions(&self) -> Vec> { - vec![self.expr.clone()] - } - - fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { - (!self.ordering_req.is_empty()).then_some(&self.ordering_req) - } - - fn order_sensitivity(&self) -> AggregateOrderSensitivity { - // Can generate correct result, even if the ordering requirement is not satisfied at the input - // (less efficient compared to ordering satisfied version) - AggregateOrderSensitivity::Beneficial - } - - fn with_requirement_satisfied( - self: Arc, - requirement_satisfied: bool, - ) -> Result>> { - let mut new_self = self.as_ref().clone(); - new_self.requirement_satisfied = requirement_satisfied; - Ok(Some(Arc::new(new_self))) - } - - fn name(&self) -> &str { - &self.name - } - - fn reverse_expr(&self) -> Option> { - Some(Arc::new(self.clone().convert_to_first())) - } - - fn create_sliding_accumulator(&self) -> Result> { - LastValueAccumulator::try_new( - &self.input_data_type, - &self.order_by_data_types, - self.ordering_req.clone(), - self.ignore_nulls, - ) - .map(|acc| { - Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ - }) - } -} - -impl PartialEq for LastValuePhysicalExpr { - fn eq(&self, other: &dyn Any) -> bool { - down_cast_any_ref(other) - .downcast_ref::() - .map(|x| { - self.name == x.name - && self.input_data_type == x.input_data_type - && self.order_by_data_types == x.order_by_data_types - && self.expr.eq(&x.expr) - }) - .unwrap_or(false) + fn aliases(&self) -> &[String] { + &self.aliases } } diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 8016b76889f7..0b35a2a1575f 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -66,11 +66,15 @@ use std::sync::Arc; /// Fluent-style API for creating `Expr`s pub mod expr_fn { pub use super::first_last::first_value; + pub use super::first_last::last_value; } /// Registers all enabled packages with a [`FunctionRegistry`] pub fn register_all(registry: &mut dyn FunctionRegistry) -> Result<()> { - let functions: Vec> = vec![first_last::first_value_udaf()]; + let functions: Vec> = vec![ + first_last::first_value_udaf(), + first_last::last_value_udaf(), + ]; functions.into_iter().try_for_each(|udf| { let existing_udaf = registry.register_udaf(udf)?; diff --git a/datafusion/optimizer/Cargo.toml b/datafusion/optimizer/Cargo.toml index b1a6953501a6..7dd9bdc2bb56 100644 --- a/datafusion/optimizer/Cargo.toml +++ b/datafusion/optimizer/Cargo.toml @@ -45,6 +45,7 @@ async-trait = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } +datafusion-functions-aggregate = { workspace = true } datafusion-physical-expr = { workspace = true } hashbrown = { version = "0.14", features = ["raw"] } itertools = { workspace = true } diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index 4f68e2623f40..683962d270ba 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -21,11 +21,9 @@ use crate::{OptimizerConfig, OptimizerRule}; use datafusion_common::{Column, Result}; use datafusion_expr::utils::expand_wildcard; -use datafusion_expr::{ - aggregate_function::AggregateFunction as AggregateFunctionFunc, col, - expr::AggregateFunction, LogicalPlanBuilder, -}; +use datafusion_expr::{col, LogicalPlanBuilder}; use datafusion_expr::{Aggregate, Distinct, DistinctOn, Expr, LogicalPlan}; +use datafusion_functions_aggregate::expr_fn::first_value; /// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] /// @@ -92,14 +90,7 @@ impl OptimizerRule for ReplaceDistinctWithAggregate { let aggr_expr = select_expr .iter() .map(|e| { - Expr::AggregateFunction(AggregateFunction::new( - AggregateFunctionFunc::FirstValue, - vec![e.clone()], - false, - None, - sort_expr.clone(), - None, - )) + first_value(vec![e.clone()], false, None, sort_expr.clone(), None) }) .collect::>(); diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index c549e6219375..fca24ce2f15c 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -360,27 +360,6 @@ pub fn create_aggregate_expr( (AggregateFunction::Median, true) => { return not_impl_err!("MEDIAN(DISTINCT) aggregations are not available"); } - (AggregateFunction::FirstValue, _) => Arc::new( - expressions::FirstValue::new( - input_phy_exprs[0].clone(), - name, - input_phy_types[0].clone(), - ordering_req.to_vec(), - ordering_types, - vec![], - ) - .with_ignore_nulls(ignore_nulls), - ), - (AggregateFunction::LastValue, _) => Arc::new( - expressions::LastValue::new( - input_phy_exprs[0].clone(), - name, - input_phy_types[0].clone(), - ordering_req.to_vec(), - ordering_types, - ) - .with_ignore_nulls(ignore_nulls), - ), (AggregateFunction::NthValue, _) => { let expr = &input_phy_exprs[0]; let Some(n) = input_phy_exprs[1] diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 260610f23dc6..ba471c76945e 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{internal_err, Result}; +use datafusion_common::{Result}; use crate::expressions::Column; use crate::PhysicalExpr; @@ -67,10 +67,10 @@ impl ProjectionMapping { // Conceptually, `source_expr` and `expression` should be the same. let idx = col.index(); let matching_input_field = input_schema.field(idx); - if col.name() != matching_input_field.name() { - return internal_err!("Input field name {} does not match with the projection expression {}", - matching_input_field.name(),col.name()) - } + // if col.name() != matching_input_field.name() { + // return internal_err!("Input field name {} does not match with the projection expression {}", + // matching_input_field.name(),col.name()) + // } let matching_input_column = Column::new(matching_input_field.name(), idx); Ok(Transformed::yes(Arc::new(matching_input_column))) diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 688d5ce6eabf..4e746b870114 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -75,9 +75,7 @@ pub use crate::window::rank::{dense_rank, percent_rank, rank}; pub use crate::window::rank::{Rank, RankType}; pub use crate::window::row_number::RowNumber; pub use crate::PhysicalSortExpr; -pub use datafusion_functions_aggregate::first_last::{ - FirstValuePhysicalExpr as FirstValue, LastValuePhysicalExpr as LastValue, -}; +pub use datafusion_functions_aggregate::first_last::{FirstValue, LastValue}; pub use binary::{binary, BinaryExpr}; pub use case::{case, CaseExpr}; From ca56d38f2e2b593a940240a889cc27ef315d75c5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 10:09:15 +0300 Subject: [PATCH 48/77] Update display of aggregate fun exprs --- datafusion/functions-aggregate/src/first_last.rs | 2 -- datafusion/physical-expr-common/src/aggregate/mod.rs | 7 ++++++- datafusion/physical-expr/src/equivalence/projection.rs | 2 +- datafusion/sqllogictest/test_files/group_by.slt | 8 ++++---- 4 files changed, 11 insertions(+), 8 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 4d46e22d2c46..dc515840bb18 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -19,7 +19,6 @@ use std::any::Any; use std::fmt::Debug; -use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray}; use arrow::compute::{self, lexsort_to_indices, SortColumn, SortOptions}; @@ -38,7 +37,6 @@ use datafusion_expr::{ use datafusion_physical_expr_common::aggregate::utils::get_sort_options; use datafusion_physical_expr_common::expressions; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_expr_common::utils::reverse_order_bys; use sqlparser::ast::NullTreatment; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 1e071f092f6d..b99a8065ca7d 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -59,12 +59,17 @@ pub fn create_aggregate_expr( .collect::>>()?; let ordering_fields = ordering_fields(ordering_req, &ordering_types); + let mut name = name.into(); + if !ordering_req.is_empty(){ + let reqs = ordering_req.iter().map(|sort_expr| format!("{sort_expr}")).collect::>(); + name = format!("{name} ORDER BY [{}]", reqs.join(", ")); + } Ok(Arc::new(AggregateFunctionExpr { fun: fun.clone(), args: input_phy_exprs.to_vec(), data_type: fun.return_type(&input_exprs_types)?, - name: name.into(), + name, schema: schema.clone(), sort_exprs: sort_exprs.to_vec(), ordering_req: ordering_req.to_vec(), diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index ba471c76945e..4dca12a81e74 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{Result}; +use datafusion_common::Result; use crate::expressions::Column; use crate::PhysicalExpr; diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index e015f7b01d0c..c97a9f01b4d5 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2016,10 +2016,10 @@ physical_plan 01)SortPreservingMergeExec: [col0@0 ASC NULLS LAST] 02)--SortExec: expr=[col0@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [col0@3 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] +07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [col0@3 ASC NULLS LAST]] 08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2178,7 +2178,7 @@ logical_plan 03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [a@0 DESC]], ordering_mode=Sorted 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2204,7 +2204,7 @@ logical_plan 03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [a@0 DESC]], ordering_mode=Sorted 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III From 5012fbded923411c98e4b350a3bab41b00c0c927 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 11:07:18 +0300 Subject: [PATCH 49/77] TMP --- datafusion/expr/src/expr.rs | 9 +++ datafusion/expr/src/udaf.rs | 28 +++++++ datafusion/expr/src/utils.rs | 29 +++++++ .../functions-aggregate/src/first_last.rs | 45 ++++++++++- .../physical-expr-common/src/aggregate/mod.rs | 76 ++++++++++++------- datafusion/physical-expr-common/src/utils.rs | 16 ++++ .../src/aggregate/array_agg_ordered.rs | 2 +- .../physical-expr/src/aggregate/nth_value.rs | 2 +- 8 files changed, 175 insertions(+), 32 deletions(-) diff --git a/datafusion/expr/src/expr.rs b/datafusion/expr/src/expr.rs index 0d8e8d816b33..8c4425b0e27e 100644 --- a/datafusion/expr/src/expr.rs +++ b/datafusion/expr/src/expr.rs @@ -550,6 +550,15 @@ impl Sort { nulls_first, } } + + /// Create a new Sort expression with the opposite sort direction + pub fn reverse(&self) -> Self { + Self { + expr: self.expr.clone(), + asc: !self.asc, + nulls_first: !self.nulls_first, + } + } } #[derive(Debug, Clone, PartialEq, Eq, Hash)] diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 67c3b51ca373..450cb51b3354 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -22,6 +22,7 @@ use crate::groups_accumulator::GroupsAccumulator; use crate::utils::format_state_name; use crate::{Accumulator, Expr}; use crate::{AccumulatorFactoryFunction, ReturnTypeFunction, Signature}; +use crate::utils::AggregateOrderSensitivity; use arrow::datatypes::{DataType, Field}; use datafusion_common::{not_impl_err, Result}; use std::any::Any; @@ -195,6 +196,19 @@ impl AggregateUDF { pub fn create_groups_accumulator(&self) -> Result> { self.inner.create_groups_accumulator() } + + pub fn with_requirement_satisfied(self: Self, requirement_satisfied: bool) -> Result> { + let updated_udf = self.inner.with_requirement_satisfied(requirement_satisfied)?; + Ok(updated_udf.map(|udf| Self{inner: udf})) + } + + pub fn order_sensitivity(&self) -> AggregateOrderSensitivity{ + self.inner.order_sensitivity() + } + + pub fn reverse_udf(&self) -> Option{ + self.inner.reverse_udf().map(|reverse| Self{inner:reverse}) + } } impl From for AggregateUDF @@ -354,6 +368,20 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn aliases(&self) -> &[String] { &[] } + + fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { + // By default, no support for this optimization + Ok(None) + } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity{ + // By default, requirement is hard if not specified. + AggregateOrderSensitivity::HardRequirement + } + + fn reverse_udf(&self) -> Option>{ + None + } } /// AggregateUDF that adds an alias to the underlying function. It is better to diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 64fe98c23b08..dc1426eae416 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1241,6 +1241,35 @@ pub fn format_state_name(name: &str, state_name: &str) -> String { format!("{name}[{state_name}]") } +/// Represents the sensitivity of an aggregate expression to ordering. +#[derive(Debug, PartialEq, Eq, Clone, Copy)] +pub enum AggregateOrderSensitivity { + /// Indicates that the aggregate expression is insensitive to ordering. Ordering at the input + /// is not important for the result of the aggregator + Insensitive, + /// Indicates that the aggregate expression has a hard requirement on ordering. Aggregator cannot produce + /// correct result unless its ordering requirement is satisfied. + HardRequirement, + /// Indicates that ordering is beneficial for the aggregate expression. Aggregator can produce its result efficiently + /// when its required ordering is satisfied. However, it can still produce correct result (less efficiently) + /// when its required ordering is not satisfied. + Beneficial, +} + +impl AggregateOrderSensitivity { + pub fn is_order_insensitive(&self) -> bool { + self.eq(&AggregateOrderSensitivity::Insensitive) + } + + pub fn is_order_beneficial(&self) -> bool { + self.eq(&AggregateOrderSensitivity::Beneficial) + } + + pub fn is_order_hard_required(&self) -> bool { + self.eq(&AggregateOrderSensitivity::HardRequirement) + } +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index dc515840bb18..e66b38686697 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -19,6 +19,7 @@ use std::any::Any; use std::fmt::Debug; +use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray}; use arrow::compute::{self, lexsort_to_indices, SortColumn, SortOptions}; @@ -29,7 +30,7 @@ use datafusion_common::{ }; use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::type_coercion::aggregates::NUMERICS; -use datafusion_expr::utils::format_state_name; +use datafusion_expr::utils::{AggregateOrderSensitivity, format_state_name}; use datafusion_expr::{ Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, Signature, TypeSignature, Volatility, @@ -50,6 +51,7 @@ make_udaf_function!( pub struct FirstValue { signature: Signature, aliases: Vec, + requirement_satisfied: bool, } impl Debug for FirstValue { @@ -80,8 +82,14 @@ impl FirstValue { ], Volatility::Immutable, ), + requirement_satisfied: false, } } + + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self{ + self.requirement_satisfied = requirement_satisfied; + self + } } impl AggregateUDFImpl for FirstValue { @@ -132,7 +140,9 @@ impl AggregateUDFImpl for FirstValue { .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; - let requirement_satisfied = ordering_req.is_empty(); + // When requirement is empty, or it is signalled by outside caller: + // accumulator assumes ordering requirement is satisfied. + let requirement_satisfied = ordering_req.is_empty() || self.requirement_satisfied; FirstValueAccumulator::try_new( acc_args.data_type, @@ -162,6 +172,18 @@ impl AggregateUDFImpl for FirstValue { fn aliases(&self) -> &[String] { &self.aliases } + + fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { + Ok(Some(Arc::new(FirstValue::new().with_requirement_satisfied(requirement_satisfied)))) + } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::Beneficial + } + + fn reverse_udf(&self) -> Option> { + Some(Arc::new(LastValue::new())) + } } #[derive(Debug)] @@ -731,6 +753,7 @@ make_udaf_function!( pub struct LastValue { signature: Signature, aliases: Vec, + requirement_satisfied: bool, } impl Debug for LastValue { @@ -761,8 +784,14 @@ impl LastValue { ], Volatility::Immutable, ), + requirement_satisfied: false, } } + + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self{ + self.requirement_satisfied = requirement_satisfied; + self + } } impl AggregateUDFImpl for LastValue { @@ -843,6 +872,18 @@ impl AggregateUDFImpl for LastValue { fn aliases(&self) -> &[String] { &self.aliases } + + fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { + Ok(Some(Arc::new(LastValue::new().with_requirement_satisfied(requirement_satisfied)))) + } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + AggregateOrderSensitivity::Beneficial + } + + fn reverse_udf(&self) -> Option> { + Some(Arc::new(FirstValue::new())) + } } #[derive(Debug)] diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index b99a8065ca7d..be3933ca7889 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -23,11 +23,13 @@ use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; +use datafusion_expr::utils::AggregateOrderSensitivity; use std::fmt::Debug; use std::{any::Any, sync::Arc}; use crate::physical_expr::PhysicalExpr; use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; +use crate::utils::{reverse_order_bys, reverse_sort_exprs}; use self::utils::{down_cast_any_ref, ordering_fields}; @@ -78,34 +80,6 @@ pub fn create_aggregate_expr( })) } -/// Represents the sensitivity of an aggregate expression to ordering. -#[derive(Debug, PartialEq, Eq, Clone, Copy)] -pub enum AggregateOrderSensitivity { - /// Indicates that the aggregate expression is insensitive to ordering. Ordering at the input - /// is not important for the result of the aggregator - Insensitive, - /// Indicates that the aggregate expression has a hard requirement on ordering. Aggregator cannot produce - /// correct result unless its ordering requirement is satisfied. - HardRequirement, - /// Indicates that ordering is beneficial for the aggregate expression. Aggregator can produce its result efficiently - /// when its required ordering is satisfied. However, it can still produce correct result (less efficiently) - /// when its required ordering is not satisfied. - Beneficial, -} -impl AggregateOrderSensitivity { - pub fn is_order_insensitive(&self) -> bool { - self.eq(&AggregateOrderSensitivity::Insensitive) - } - - pub fn is_order_beneficial(&self) -> bool { - self.eq(&AggregateOrderSensitivity::Beneficial) - } - - pub fn is_order_hard_required(&self) -> bool { - self.eq(&AggregateOrderSensitivity::HardRequirement) - } -} - /// An aggregate expression that: /// * knows its resulting field /// * knows how to create its accumulator @@ -335,6 +309,52 @@ impl AggregateExpr for AggregateFunctionExpr { fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { (!self.ordering_req.is_empty()).then_some(&self.ordering_req) } + + fn order_sensitivity(&self) -> AggregateOrderSensitivity { + if !self.ordering_req.is_empty(){ + // If there is requirement, use the sensitive of the implementation + self.fun.order_sensitivity() + } else { + // If no requirement, aggregator is order insensitive + AggregateOrderSensitivity::Insensitive + } + } + + fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { + if let Some(updated_fn) = self.fun.clone().with_requirement_satisfied(requirement_satisfied)?{ + return Ok(Some(Arc::new(AggregateFunctionExpr{ + fun: updated_fn, + args: self.args.clone(), + data_type: self.data_type.clone(), + name: self.fun.name().to_string(), + schema: self.schema.clone(), + sort_exprs: self.sort_exprs.clone(), + ordering_req: self.ordering_req.clone(), + ignore_nulls: self.ignore_nulls, + ordering_fields: self.ordering_fields.clone(), + }))); + } + Ok(None) + } + + fn reverse_expr(&self) -> Option> { + if let Some(reverse_udf) = self.fun.reverse_udf(){ + let reverse_ordering_req = reverse_order_bys(&self.ordering_req); + let reverse_sort_exprs = reverse_sort_exprs(&self.sort_exprs); + return Some(Arc::new(AggregateFunctionExpr{ + fun: reverse_udf, + args: self.args.clone(), + data_type: self.data_type.clone(), + name: self.fun.name().to_string(), + schema: self.schema.clone(), + sort_exprs: reverse_sort_exprs, + ordering_req: reverse_ordering_req, + ignore_nulls: self.ignore_nulls, + ordering_fields: self.ordering_fields.clone(), + })) + } + None + } } impl PartialEq for AggregateFunctionExpr { diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 459b5a4849cb..85fe8ba5a93c 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -20,6 +20,7 @@ use arrow::{ compute::{and_kleene, is_not_null, SlicesIterator}, }; use datafusion_common::Result; +use datafusion_expr::Expr; use crate::sort_expr::PhysicalSortExpr; @@ -79,6 +80,21 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec Vec { + sort_exprs + .iter() + .map(|e| { + if let Expr::Sort(s) = e { + Expr::Sort(s.reverse()) + } else { + // TODO: Return error + unreachable!() + } + }) + .collect::>() +} + #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 7b6fdc8cdeaf..df63fcc49e59 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -40,7 +40,7 @@ use datafusion_common::utils::array_into_list_array; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::Accumulator; -use datafusion_physical_expr_common::aggregate::AggregateOrderSensitivity; +use datafusion_expr::utils::AggregateOrderSensitivity; /// Expression for a `ARRAY_AGG(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, diff --git a/datafusion/physical-expr/src/aggregate/nth_value.rs b/datafusion/physical-expr/src/aggregate/nth_value.rs index ae46aba76152..eac32640dad2 100644 --- a/datafusion/physical-expr/src/aggregate/nth_value.rs +++ b/datafusion/physical-expr/src/aggregate/nth_value.rs @@ -35,7 +35,7 @@ use arrow_schema::{DataType, Field, Fields}; use datafusion_common::utils::{array_into_list_array, get_row_at_idx}; use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; use datafusion_expr::Accumulator; -use datafusion_physical_expr_common::aggregate::AggregateOrderSensitivity; +use datafusion_expr::utils::AggregateOrderSensitivity; /// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, From fef594e957d96e4b95ed790afdf95e777ba4eafa Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 11:13:59 +0300 Subject: [PATCH 50/77] TMP --- datafusion/expr/src/udaf.rs | 30 ++++++++---- .../functions-aggregate/src/first_last.rs | 26 ++++++---- .../physical-expr-common/src/aggregate/mod.rs | 48 +++++++++++-------- .../src/aggregate/array_agg_ordered.rs | 2 +- .../physical-expr/src/aggregate/nth_value.rs | 2 +- 5 files changed, 69 insertions(+), 39 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 450cb51b3354..22ab3f249010 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -20,9 +20,9 @@ use crate::function::AccumulatorArgs; use crate::groups_accumulator::GroupsAccumulator; use crate::utils::format_state_name; +use crate::utils::AggregateOrderSensitivity; use crate::{Accumulator, Expr}; use crate::{AccumulatorFactoryFunction, ReturnTypeFunction, Signature}; -use crate::utils::AggregateOrderSensitivity; use arrow::datatypes::{DataType, Field}; use datafusion_common::{not_impl_err, Result}; use std::any::Any; @@ -197,17 +197,24 @@ impl AggregateUDF { self.inner.create_groups_accumulator() } - pub fn with_requirement_satisfied(self: Self, requirement_satisfied: bool) -> Result> { - let updated_udf = self.inner.with_requirement_satisfied(requirement_satisfied)?; - Ok(updated_udf.map(|udf| Self{inner: udf})) + pub fn with_requirement_satisfied( + self: Self, + requirement_satisfied: bool, + ) -> Result> { + let updated_udf = self + .inner + .with_requirement_satisfied(requirement_satisfied)?; + Ok(updated_udf.map(|udf| Self { inner: udf })) } - pub fn order_sensitivity(&self) -> AggregateOrderSensitivity{ + pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { self.inner.order_sensitivity() } - pub fn reverse_udf(&self) -> Option{ - self.inner.reverse_udf().map(|reverse| Self{inner:reverse}) + pub fn reverse_udf(&self) -> Option { + self.inner + .reverse_udf() + .map(|reverse| Self { inner: reverse }) } } @@ -369,17 +376,20 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { &[] } - fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { + fn with_requirement_satisfied( + self: Arc, + requirement_satisfied: bool, + ) -> Result>> { // By default, no support for this optimization Ok(None) } - fn order_sensitivity(&self) -> AggregateOrderSensitivity{ + fn order_sensitivity(&self) -> AggregateOrderSensitivity { // By default, requirement is hard if not specified. AggregateOrderSensitivity::HardRequirement } - fn reverse_udf(&self) -> Option>{ + fn reverse_udf(&self) -> Option> { None } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index e66b38686697..30cdb025e797 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -30,7 +30,7 @@ use datafusion_common::{ }; use datafusion_expr::function::AccumulatorArgs; use datafusion_expr::type_coercion::aggregates::NUMERICS; -use datafusion_expr::utils::{AggregateOrderSensitivity, format_state_name}; +use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; use datafusion_expr::{ Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, Signature, TypeSignature, Volatility, @@ -86,7 +86,7 @@ impl FirstValue { } } - fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self{ + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { self.requirement_satisfied = requirement_satisfied; self } @@ -173,8 +173,13 @@ impl AggregateUDFImpl for FirstValue { &self.aliases } - fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { - Ok(Some(Arc::new(FirstValue::new().with_requirement_satisfied(requirement_satisfied)))) + fn with_requirement_satisfied( + self: Arc, + requirement_satisfied: bool, + ) -> Result>> { + Ok(Some(Arc::new( + FirstValue::new().with_requirement_satisfied(requirement_satisfied), + ))) } fn order_sensitivity(&self) -> AggregateOrderSensitivity { @@ -788,7 +793,7 @@ impl LastValue { } } - fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self{ + fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { self.requirement_satisfied = requirement_satisfied; self } @@ -873,14 +878,19 @@ impl AggregateUDFImpl for LastValue { &self.aliases } - fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { - Ok(Some(Arc::new(LastValue::new().with_requirement_satisfied(requirement_satisfied)))) + fn with_requirement_satisfied( + self: Arc, + requirement_satisfied: bool, + ) -> Result>> { + Ok(Some(Arc::new( + LastValue::new().with_requirement_satisfied(requirement_satisfied), + ))) } fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Beneficial } - + fn reverse_udf(&self) -> Option> { Some(Arc::new(FirstValue::new())) } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index be3933ca7889..3cfda0306b27 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -20,10 +20,10 @@ pub mod utils; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::{exec_datafusion_err, not_impl_err, Result}; use datafusion_expr::type_coercion::aggregates::check_arg_count; +use datafusion_expr::utils::AggregateOrderSensitivity; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; -use datafusion_expr::utils::AggregateOrderSensitivity; use std::fmt::Debug; use std::{any::Any, sync::Arc}; @@ -62,8 +62,11 @@ pub fn create_aggregate_expr( let ordering_fields = ordering_fields(ordering_req, &ordering_types); let mut name = name.into(); - if !ordering_req.is_empty(){ - let reqs = ordering_req.iter().map(|sort_expr| format!("{sort_expr}")).collect::>(); + if !ordering_req.is_empty() { + let reqs = ordering_req + .iter() + .map(|sort_expr| format!("{sort_expr}")) + .collect::>(); name = format!("{name} ORDER BY [{}]", reqs.join(", ")); } @@ -311,7 +314,7 @@ impl AggregateExpr for AggregateFunctionExpr { } fn order_sensitivity(&self) -> AggregateOrderSensitivity { - if !self.ordering_req.is_empty(){ + if !self.ordering_req.is_empty() { // If there is requirement, use the sensitive of the implementation self.fun.order_sensitivity() } else { @@ -320,9 +323,16 @@ impl AggregateExpr for AggregateFunctionExpr { } } - fn with_requirement_satisfied(self: Arc, requirement_satisfied: bool) -> Result>> { - if let Some(updated_fn) = self.fun.clone().with_requirement_satisfied(requirement_satisfied)?{ - return Ok(Some(Arc::new(AggregateFunctionExpr{ + fn with_requirement_satisfied( + self: Arc, + requirement_satisfied: bool, + ) -> Result>> { + if let Some(updated_fn) = self + .fun + .clone() + .with_requirement_satisfied(requirement_satisfied)? + { + return Ok(Some(Arc::new(AggregateFunctionExpr { fun: updated_fn, args: self.args.clone(), data_type: self.data_type.clone(), @@ -338,20 +348,20 @@ impl AggregateExpr for AggregateFunctionExpr { } fn reverse_expr(&self) -> Option> { - if let Some(reverse_udf) = self.fun.reverse_udf(){ + if let Some(reverse_udf) = self.fun.reverse_udf() { let reverse_ordering_req = reverse_order_bys(&self.ordering_req); let reverse_sort_exprs = reverse_sort_exprs(&self.sort_exprs); - return Some(Arc::new(AggregateFunctionExpr{ - fun: reverse_udf, - args: self.args.clone(), - data_type: self.data_type.clone(), - name: self.fun.name().to_string(), - schema: self.schema.clone(), - sort_exprs: reverse_sort_exprs, - ordering_req: reverse_ordering_req, - ignore_nulls: self.ignore_nulls, - ordering_fields: self.ordering_fields.clone(), - })) + let reverse_aggr = create_aggregate_expr( + &reverse_udf, + &self.args, + &reverse_sort_exprs, + &reverse_ordering_req, + &self.schema, + self.fun.name(), + self.ignore_nulls, + ) + .unwrap(); + return Some(reverse_aggr); } None } diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index df63fcc49e59..78eac79f20f8 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -39,8 +39,8 @@ use arrow_schema::{Fields, SortOptions}; use datafusion_common::utils::array_into_list_array; use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, Result, ScalarValue}; -use datafusion_expr::Accumulator; use datafusion_expr::utils::AggregateOrderSensitivity; +use datafusion_expr::Accumulator; /// Expression for a `ARRAY_AGG(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, diff --git a/datafusion/physical-expr/src/aggregate/nth_value.rs b/datafusion/physical-expr/src/aggregate/nth_value.rs index eac32640dad2..ee7426a897b3 100644 --- a/datafusion/physical-expr/src/aggregate/nth_value.rs +++ b/datafusion/physical-expr/src/aggregate/nth_value.rs @@ -34,8 +34,8 @@ use arrow_array::{new_empty_array, ArrayRef, StructArray}; use arrow_schema::{DataType, Field, Fields}; use datafusion_common::utils::{array_into_list_array, get_row_at_idx}; use datafusion_common::{exec_err, internal_err, Result, ScalarValue}; -use datafusion_expr::Accumulator; use datafusion_expr::utils::AggregateOrderSensitivity; +use datafusion_expr::Accumulator; /// Expression for a `NTH_VALUE(... ORDER BY ..., ...)` aggregation. In a multi /// partition setting, partial aggregations are computed for every partition, From 25ba8e8ba5431972948fb3cc6f2415866e601394 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 13:22:50 +0300 Subject: [PATCH 51/77] Update tests --- datafusion/expr/src/udaf.rs | 4 +- .../physical-expr-common/src/aggregate/mod.rs | 34 +++++++++----- .../sqllogictest/test_files/aggregate.slt | 8 ++-- .../sqllogictest/test_files/distinct_on.slt | 4 +- .../sqllogictest/test_files/group_by.slt | 44 +++++++++---------- datafusion/sqllogictest/test_files/joins.slt | 10 ++--- 6 files changed, 57 insertions(+), 47 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 22ab3f249010..4dc75e314a8f 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -198,7 +198,7 @@ impl AggregateUDF { } pub fn with_requirement_satisfied( - self: Self, + self, requirement_satisfied: bool, ) -> Result> { let updated_udf = self @@ -378,7 +378,7 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { fn with_requirement_satisfied( self: Arc, - requirement_satisfied: bool, + _requirement_satisfied: bool, ) -> Result>> { // By default, no support for this optimization Ok(None) diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 3cfda0306b27..95364898fe07 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -83,6 +83,14 @@ pub fn create_aggregate_expr( })) } +fn calc_fn_name_with_args(fn_name: &str, args: &[Arc]) -> String { + let args = args + .iter() + .map(|sort_expr| format!("{sort_expr}")) + .collect::>(); + format!("{fn_name}({})", args.join(", ")) +} + /// An aggregate expression that: /// * knows its resulting field /// * knows how to create its accumulator @@ -332,17 +340,18 @@ impl AggregateExpr for AggregateFunctionExpr { .clone() .with_requirement_satisfied(requirement_satisfied)? { - return Ok(Some(Arc::new(AggregateFunctionExpr { - fun: updated_fn, - args: self.args.clone(), - data_type: self.data_type.clone(), - name: self.fun.name().to_string(), - schema: self.schema.clone(), - sort_exprs: self.sort_exprs.clone(), - ordering_req: self.ordering_req.clone(), - ignore_nulls: self.ignore_nulls, - ordering_fields: self.ordering_fields.clone(), - }))); + let name = calc_fn_name_with_args(self.fun.name(), &self.args); + let aggr_expr = create_aggregate_expr( + &updated_fn, + &self.args, + &self.sort_exprs, + &self.ordering_req, + &self.schema, + name, + self.ignore_nulls, + ) + .unwrap(); + return Ok(Some(aggr_expr)); } Ok(None) } @@ -351,13 +360,14 @@ impl AggregateExpr for AggregateFunctionExpr { if let Some(reverse_udf) = self.fun.reverse_udf() { let reverse_ordering_req = reverse_order_bys(&self.ordering_req); let reverse_sort_exprs = reverse_sort_exprs(&self.sort_exprs); + let name = calc_fn_name_with_args(self.fun.name(), &self.args); let reverse_aggr = create_aggregate_expr( &reverse_udf, &self.args, &reverse_sort_exprs, &reverse_ordering_req, &self.schema, - self.fun.name(), + name, self.ignore_nulls, ) .unwrap(); diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 8b5b84e76650..359a949fe760 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3520,9 +3520,9 @@ logical_plan 01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] 02)--TableScan: convert_first_last_table projection=[c1, c3] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]] +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [c3@1 DESC]] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(c1@0) ORDER BY [c3@1 ASC NULLS LAST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true @@ -3534,8 +3534,8 @@ logical_plan 01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] 02)--TableScan: convert_first_last_table projection=[c1, c2] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]] +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [c2@1 ASC NULLS LAST]] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(c1@0) ORDER BY [c2@1 DESC]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 972c935cee99..d8af4646f55a 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -97,10 +97,10 @@ physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] 02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 03)----SortExec: expr=[c1@0 ASC NULLS LAST] -04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(c3@2) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST], FIRST_VALUE(c2@1) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index c97a9f01b4d5..e3a884d3d809 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2019,7 +2019,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [col0@3 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [col0@3 ASC NULLS LAST]] +07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(col1@4) ORDER BY [col0@3 ASC NULLS LAST]] 08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2178,7 +2178,7 @@ logical_plan 03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [a@0 DESC]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(c@2) ORDER BY [a@0 DESC]], ordering_mode=Sorted 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2204,7 +2204,7 @@ logical_plan 03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [a@0 DESC]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(c@2) ORDER BY [a@0 DESC]], ordering_mode=Sorted 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2677,7 +2677,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC]] 03)----SortExec: expr=[amount@1 DESC] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2708,7 +2708,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST], FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2740,7 +2740,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST], FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2805,7 +2805,7 @@ logical_plan 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 DESC], LAST_VALUE(amount@2) ORDER BY [ts@1 DESC], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2838,7 +2838,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, ts, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 DESC], LAST_VALUE(amount@2) ORDER BY [ts@1 DESC], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2874,7 +2874,7 @@ logical_plan physical_plan 01)SortExec: expr=[sn@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] -03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]] +03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(amount@6) ORDER BY [sn@5 ASC NULLS LAST]] 04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] @@ -2919,11 +2919,11 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@1 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(amount@2) ORDER BY [ts@1 ASC NULLS LAST]] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2955,11 +2955,11 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@1 DESC]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(amount@2) ORDER BY [ts@1 DESC]] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2991,9 +2991,9 @@ logical_plan 03)----TableScan: sales_global projection=[ts, amount] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] -02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@0 ASC NULLS LAST]] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(amount@1) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(amount@1) ORDER BY [ts@0 ASC NULLS LAST]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3017,9 +3017,9 @@ logical_plan 03)----TableScan: sales_global projection=[ts, amount] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] -02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@0 DESC]] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(amount@1) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(amount@1) ORDER BY [ts@0 DESC]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3154,10 +3154,10 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [amount@1 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [amount@1 DESC]] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC]] 08)--------------SortExec: expr=[amount@1 DESC] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3800,10 +3800,10 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [a@0 ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [c@1 DESC]] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 -05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] +05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(a@0) ORDER BY [a@0 ASC NULLS LAST], FIRST_VALUE(c@1) ORDER BY [c@1 ASC NULLS LAST]] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -3870,7 +3870,7 @@ logical_plan 12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] -02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(d@1) ORDER BY [a@0 ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 5ef33307b521..bd99e0336f68 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2010,7 +2010,7 @@ set datafusion.explain.logical_plan_only = false; statement ok set datafusion.execution.target_partitions = 4; -# Planning inner nested loop join +# Planning inner nested loop join # inputs are swapped due to inexact statistics + join reordering caused additional projection query TT @@ -3362,7 +3362,7 @@ logical_plan 08)----------TableScan: annotated_data projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(b@4) ORDER BY [a@3 ASC]], ordering_mode=PartiallySorted([0]) 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -3410,7 +3410,7 @@ logical_plan 12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] -02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(d@1) ORDER BY [a@0 ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true @@ -3447,10 +3447,10 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC] 02)--SortExec: expr=[a@0 ASC] 03)----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [a@3 ASC]] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] +07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(b@4) ORDER BY [a@3 ASC]] 08)--------------CoalesceBatchesExec: target_batch_size=2 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] 10)------------------CoalesceBatchesExec: target_batch_size=2 From a530dc4b768a3cd52695d4fe66b97f67846c4f43 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 14:11:00 +0300 Subject: [PATCH 52/77] TMP buggy --- .../functions-aggregate/src/first_last.rs | 12 ++- .../src/equivalence/projection.rs | 10 +- .../sqllogictest/test_files/distinct_on.slt | 93 +++++++++++++++++++ 3 files changed, 106 insertions(+), 9 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 30cdb025e797..85574ff5e7e8 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -87,7 +87,8 @@ impl FirstValue { } fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; + // self.requirement_satisfied = requirement_satisfied; + self.requirement_satisfied = false; self } } @@ -232,7 +233,8 @@ impl FirstValueAccumulator { } pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; + // self.requirement_satisfied = requirement_satisfied; + self.requirement_satisfied = false; self } @@ -794,7 +796,8 @@ impl LastValue { } fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; + // self.requirement_satisfied = requirement_satisfied; + self.requirement_satisfied = false; self } } @@ -989,7 +992,8 @@ impl LastValueAccumulator { } fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - self.requirement_satisfied = requirement_satisfied; + // self.requirement_satisfied = requirement_satisfied; + self.requirement_satisfied = false; self } } diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 4dca12a81e74..260610f23dc6 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::Result; +use datafusion_common::{internal_err, Result}; use crate::expressions::Column; use crate::PhysicalExpr; @@ -67,10 +67,10 @@ impl ProjectionMapping { // Conceptually, `source_expr` and `expression` should be the same. let idx = col.index(); let matching_input_field = input_schema.field(idx); - // if col.name() != matching_input_field.name() { - // return internal_err!("Input field name {} does not match with the projection expression {}", - // matching_input_field.name(),col.name()) - // } + if col.name() != matching_input_field.name() { + return internal_err!("Input field name {} does not match with the projection expression {}", + matching_input_field.name(),col.name()) + } let matching_input_column = Column::new(matching_input_field.name(), idx); Ok(Transformed::yes(Arc::new(matching_input_column))) diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index d8af4646f55a..f22c5f00ea29 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -112,6 +112,99 @@ SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 query error DataFusion error: Error during planning: No `ON` expressions provided SELECT DISTINCT ON () c1, c2 FROM aggregate_test_100 ORDER BY c1, c2; +# query BII +# SELECT c2 % 2 = 0, c2, c3 - 100 +# FROM aggregate_test_100 +# ORDER BY c2 % 2 = 0, c3 DESC; +# ---- + +query TT +explain +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +logical_plan +01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) +02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST +03)----Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] +04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, aggregate_test_100.c2, aggregate_test_100.c3 +05)--------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] +02)--SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +03)----SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +04)------AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(c2@1) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(CAST(c3@2 AS Int64) - 100) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] +08)--------------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, c2@0 as c2, c3@1 as c3] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +statement ok +set datafusion.execution.target_partitions = 1; + +query TT +explain +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +logical_plan +01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) +02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST +03)----Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] +04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, aggregate_test_100.c2, aggregate_test_100.c3 +05)--------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] +02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +03)----AggregateExec: mode=Final, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] +04)------AggregateExec: mode=Partial, gby=[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(c2@1) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(CAST(c3@2 AS Int64) - 100) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] +05)--------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, c2@0 as c2, c3@1 as c3] +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query II +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +5 -140 +2 -99 + +statement ok +set datafusion.execution.target_partitions = 4; + +query TT +EXPLAIN SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) +FROM aggregate_test_100 +GROUP BY c2 % 2 = 0 +ORDER BY c2 % 2 = 0; +---- +logical_plan +01)Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST +02)--Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]]] +03)----TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +03)----AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [c3@1 DESC]] +04)------CoalesceBatchesExec: target_batch_size=8192 +05)--------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 +06)----------AggregateExec: mode=Partial, gby=[CAST(c2@0 AS Int64) % 2 = 0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(CAST(c3@1 AS Int64) - 100) ORDER BY [c3@1 DESC]] +07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query BI +SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) +FROM aggregate_test_100 +GROUP BY c2 % 2 = 0 +ORDER BY c2 % 2 = 0; +---- +false 25 +true 23 + +query II +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +5 -140 +2 -99 + # Use expressions in the ON and ORDER BY clauses, as well as the selection query II SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; From 98320b0cd000fbd94b81085a4a7b5b577a505349 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 14:56:37 +0300 Subject: [PATCH 53/77] modify name in place --- datafusion/core/src/physical_planner.rs | 16 ++++++- .../functions-aggregate/src/first_last.rs | 8 ++-- .../physical-expr-common/src/aggregate/mod.rs | 48 ++++++++++++++----- .../sqllogictest/test_files/aggregate.slt | 8 ++-- .../sqllogictest/test_files/group_by.slt | 46 +++++++++--------- datafusion/sqllogictest/test_files/joins.slt | 8 ++-- 6 files changed, 86 insertions(+), 48 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index b7b6c20b19bb..5fdeba2a9452 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1930,6 +1930,21 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( (agg_expr, filter, physical_sort_exprs) } AggregateFunctionDefinition::UDF(fun) => { + let name = match order_by { + Some(order_by) => { + if order_by.is_empty() { + name.into() + } else { + let name = name.into(); + let reqs = order_by + .iter() + .map(|sort_expr| format!("{sort_expr}")) + .collect::>(); + format!("{name} ORDER BY [{}]", reqs.join(", ")) + } + } + None => name.into(), + }; let sort_exprs = order_by.clone().unwrap_or(vec![]); let physical_sort_exprs = match order_by { Some(exprs) => Some(create_physical_sort_exprs( @@ -1976,7 +1991,6 @@ pub fn create_aggregate_expr_and_maybe_filter( Expr::Alias(Alias { expr, name, .. }) => (name.clone(), expr.as_ref()), _ => (physical_name(e)?, e), }; - create_aggregate_expr_with_name_and_maybe_filter( e, name, diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 85574ff5e7e8..2b8b5dca5185 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -233,8 +233,7 @@ impl FirstValueAccumulator { } pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - // self.requirement_satisfied = requirement_satisfied; - self.requirement_satisfied = false; + self.requirement_satisfied = requirement_satisfied; self } @@ -850,7 +849,7 @@ impl AggregateUDFImpl for LastValue { .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; - let requirement_satisfied = ordering_req.is_empty(); + let requirement_satisfied = ordering_req.is_empty() || self.requirement_satisfied; LastValueAccumulator::try_new( acc_args.data_type, @@ -992,8 +991,7 @@ impl LastValueAccumulator { } fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - // self.requirement_satisfied = requirement_satisfied; - self.requirement_satisfied = false; + self.requirement_satisfied = requirement_satisfied; self } } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 95364898fe07..dc602fab4b52 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -61,14 +61,7 @@ pub fn create_aggregate_expr( .collect::>>()?; let ordering_fields = ordering_fields(ordering_req, &ordering_types); - let mut name = name.into(); - if !ordering_req.is_empty() { - let reqs = ordering_req - .iter() - .map(|sort_expr| format!("{sort_expr}")) - .collect::>(); - name = format!("{name} ORDER BY [{}]", reqs.join(", ")); - } + let name = name.into(); Ok(Arc::new(AggregateFunctionExpr { fun: fun.clone(), @@ -340,14 +333,14 @@ impl AggregateExpr for AggregateFunctionExpr { .clone() .with_requirement_satisfied(requirement_satisfied)? { - let name = calc_fn_name_with_args(self.fun.name(), &self.args); + // let name = calc_fn_name_with_args(self.fun.name(), &self.args); let aggr_expr = create_aggregate_expr( &updated_fn, &self.args, &self.sort_exprs, &self.ordering_req, &self.schema, - name, + self.name(), self.ignore_nulls, ) .unwrap(); @@ -360,7 +353,11 @@ impl AggregateExpr for AggregateFunctionExpr { if let Some(reverse_udf) = self.fun.reverse_udf() { let reverse_ordering_req = reverse_order_bys(&self.ordering_req); let reverse_sort_exprs = reverse_sort_exprs(&self.sort_exprs); - let name = calc_fn_name_with_args(self.fun.name(), &self.args); + let mut name = self.name().to_string(); + // let name = calc_fn_name_with_args(self.fun.name(), &self.args); + // let name = self.fun.name(); + replace_order_by_clause(&mut name); + replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); let reverse_aggr = create_aggregate_expr( &reverse_udf, &self.args, @@ -395,3 +392,32 @@ impl PartialEq for AggregateFunctionExpr { .unwrap_or(false) } } + +fn replace_order_by_clause(order_by: &mut String) { + let suffixes = [ + (" DESC NULLS FIRST]", " ASC NULLS LAST]"), + (" ASC NULLS FIRST]", " DESC NULLS LAST]"), + (" DESC NULLS LAST]", " ASC NULLS FIRST]"), + (" ASC NULLS LAST]", " DESC NULLS FIRST]"), + ]; + + if let Some(start) = order_by.find("ORDER BY [") { + if let Some(end) = order_by[start..].find(']') { + let order_by_start = start + 9; + let order_by_end = start + end; + + let column_order = &order_by[order_by_start..=order_by_end]; + for &(suffix, replacement) in &suffixes { + if column_order.ends_with(suffix) { + let new_order = column_order.replace(suffix, replacement); + order_by.replace_range(order_by_start..=order_by_end, &new_order); + break; + } + } + } + } +} + +fn replace_fn_name_clause(aggr_name: &mut String, fn_name_old: &str, fn_name_new: &str) { + *aggr_name = aggr_name.replace(fn_name_old, fn_name_new); +} diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index 359a949fe760..8b5b84e76650 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -3520,9 +3520,9 @@ logical_plan 01)Aggregate: groupBy=[[]], aggr=[[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]]] 02)--TableScan: convert_first_last_table projection=[c1, c3] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [c3@1 DESC]] +01)AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(c1@0) ORDER BY [c3@1 ASC NULLS LAST]] +03)----AggregateExec: mode=Partial, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true @@ -3534,8 +3534,8 @@ logical_plan 01)Aggregate: groupBy=[[]], aggr=[[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]]] 02)--TableScan: convert_first_last_table projection=[c1, c2] physical_plan -01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [c2@1 ASC NULLS LAST]] +01)AggregateExec: mode=Final, gby=[], aggr=[LAST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]] 02)--CoalescePartitionsExec -03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(c1@0) ORDER BY [c2@1 DESC]] +03)----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] 04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index e3a884d3d809..e015f7b01d0c 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2016,10 +2016,10 @@ physical_plan 01)SortPreservingMergeExec: [col0@0 ASC NULLS LAST] 02)--SortExec: expr=[col0@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [col0@3 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(col1@4) ORDER BY [col0@3 ASC NULLS LAST]] +07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] 08)--------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 09)----------------CoalesceBatchesExec: target_batch_size=8192 10)------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2178,7 +2178,7 @@ logical_plan 03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(c@2) ORDER BY [a@0 DESC]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2204,7 +2204,7 @@ logical_plan 03)----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan 01)ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(c@2) ORDER BY [a@0 DESC]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]], ordering_mode=Sorted 03)----StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] query III @@ -2677,7 +2677,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 03)----SortExec: expr=[amount@1 DESC] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2708,7 +2708,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST], FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2740,7 +2740,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@2 as fv2, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@3 as amounts] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST], FIRST_VALUE(amount@1) ORDER BY [amount@1 ASC NULLS LAST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 03)----SortExec: expr=[amount@1 ASC NULLS LAST] 04)------MemoryExec: partitions=1, partition_sizes=[1] @@ -2805,7 +2805,7 @@ logical_plan 04)------TableScan: sales_global projection=[country, ts, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 DESC], LAST_VALUE(amount@2) ORDER BY [ts@1 DESC], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2838,7 +2838,7 @@ logical_plan 03)----TableScan: sales_global projection=[country, ts, amount] physical_plan 01)ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as lv1, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as sum1] -02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 DESC], LAST_VALUE(amount@2) ORDER BY [ts@1 DESC], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] +02)--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST], SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort @@ -2874,7 +2874,7 @@ logical_plan physical_plan 01)SortExec: expr=[sn@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] -03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(amount@6) ORDER BY [sn@5 ASC NULLS LAST]] +03)----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]] 04)------ProjectionExec: expr=[zip_code@2 as zip_code, country@3 as country, sn@4 as sn, ts@5 as ts, currency@6 as currency, sn@0 as sn, amount@1 as amount] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@2, currency@4)], filter=ts@0 >= ts@1, projection=[sn@0, amount@3, zip_code@4, country@5, sn@6, ts@7, currency@8] @@ -2919,11 +2919,11 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@1 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(amount@2) ORDER BY [ts@1 ASC NULLS LAST]] +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2955,11 +2955,11 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@1 DESC]] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(amount@2) ORDER BY [ts@1 ASC NULLS LAST], LAST_VALUE(amount@2) ORDER BY [ts@1 DESC]] +08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2991,9 +2991,9 @@ logical_plan 03)----TableScan: sales_global projection=[ts, amount] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv2] -02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@0 ASC NULLS LAST]] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(amount@1) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(amount@1) ORDER BY [ts@0 ASC NULLS LAST]] +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3017,9 +3017,9 @@ logical_plan 03)----TableScan: sales_global projection=[ts, amount] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] -02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [ts@0 DESC]] +02)--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----CoalescePartitionsExec -04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(amount@1) ORDER BY [ts@0 ASC NULLS LAST], LAST_VALUE(amount@1) ORDER BY [ts@0 DESC]] +04)------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -3154,10 +3154,10 @@ physical_plan 01)SortPreservingMergeExec: [country@0 ASC NULLS LAST] 02)--SortExec: expr=[country@0 ASC NULLS LAST] 03)----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [amount@1 ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [amount@1 DESC]] +04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC], LAST_VALUE(amount@1) ORDER BY [amount@1 DESC]] +07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 08)--------------SortExec: expr=[amount@1 DESC] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3800,10 +3800,10 @@ logical_plan 03)----TableScan: multiple_ordered_table projection=[a, c, d] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] -02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [a@0 ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [c@1 DESC]] +02)--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 -05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(a@0) ORDER BY [a@0 ASC NULLS LAST], FIRST_VALUE(c@1) ORDER BY [c@1 ASC NULLS LAST]] +05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] 06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -3870,7 +3870,7 @@ logical_plan 12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] -02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(d@1) ORDER BY [a@0 ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index bd99e0336f68..0844cb94ace6 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3362,7 +3362,7 @@ logical_plan 08)----------TableScan: annotated_data projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(b@4) ORDER BY [a@3 ASC]], ordering_mode=PartiallySorted([0]) +02)--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]], ordering_mode=PartiallySorted([0]) 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -3410,7 +3410,7 @@ logical_plan 12)------------------TableScan: multiple_ordered_table projection=[a, d] physical_plan 01)ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd] -02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(d@1) ORDER BY [a@0 ASC NULLS LAST]], ordering_mode=Sorted +02)--AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]], ordering_mode=Sorted 03)----CoalesceBatchesExec: target_batch_size=2 04)------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10, projection=[a@0, d@1, row_n@4] 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true @@ -3447,10 +3447,10 @@ physical_plan 01)SortPreservingMergeExec: [a@0 ASC] 02)--SortExec: expr=[a@0 ASC] 03)----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] -04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [a@3 ASC]] +04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(b@4) ORDER BY [a@3 ASC]] +07)------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]] 08)--------------CoalesceBatchesExec: target_batch_size=2 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] 10)------------------CoalesceBatchesExec: target_batch_size=2 From 257abbb3eafbd194769836e04c000dca4569fa12 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 14:58:27 +0300 Subject: [PATCH 54/77] Minor changes --- .../sqllogictest/test_files/distinct_on.slt | 50 +------------------ 1 file changed, 2 insertions(+), 48 deletions(-) diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index f22c5f00ea29..932fd42180ad 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -97,10 +97,10 @@ physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@1 as c3, FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]@2 as c2] 02)--SortPreservingMergeExec: [c1@0 ASC NULLS LAST] 03)----SortExec: expr=[c1@0 ASC NULLS LAST] -04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST]] +04)------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(c3@2) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST], FIRST_VALUE(c2@1) ORDER BY [c1@0 ASC NULLS LAST, c3@2 ASC NULLS LAST]] +07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], FIRST_VALUE(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] 08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true @@ -118,58 +118,12 @@ SELECT DISTINCT ON () c1, c2 FROM aggregate_test_100 ORDER BY c1, c2; # ORDER BY c2 % 2 = 0, c3 DESC; # ---- -query TT -explain -SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ----- -logical_plan -01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) -02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST -03)----Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] -04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, aggregate_test_100.c2, aggregate_test_100.c3 -05)--------TableScan: aggregate_test_100 projection=[c2, c3] -physical_plan -01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] -02)--SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -03)----SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -04)------AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(c2@1) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(CAST(c3@2 AS Int64) - 100) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] -08)--------------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, c2@0 as c2, c3@1 as c3] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - -statement ok -set datafusion.execution.target_partitions = 1; - -query TT -explain -SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ----- -logical_plan -01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) -02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST -03)----Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] -04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, aggregate_test_100.c2, aggregate_test_100.c3 -05)--------TableScan: aggregate_test_100 projection=[c2, c3] -physical_plan -01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] -02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -03)----AggregateExec: mode=Final, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] -04)------AggregateExec: mode=Partial, gby=[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(c2@1) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC], FIRST_VALUE(CAST(c3@2 AS Int64) - 100) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 ASC NULLS LAST, c3@2 DESC]] -05)--------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, c2@0 as c2, c3@1 as c3] -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - query II SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ---- 5 -140 2 -99 -statement ok -set datafusion.execution.target_partitions = 4; - query TT EXPLAIN SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) FROM aggregate_test_100 From 0f9a903ac4c23a00267682565e639d230eb2b4a8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 25 Apr 2024 14:58:48 +0300 Subject: [PATCH 55/77] Tmp --- datafusion/sqllogictest/test_files/joins.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0844cb94ace6..5ef33307b521 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -2010,7 +2010,7 @@ set datafusion.explain.logical_plan_only = false; statement ok set datafusion.execution.target_partitions = 4; -# Planning inner nested loop join +# Planning inner nested loop join # inputs are swapped due to inexact statistics + join reordering caused additional projection query TT From b3168859891a2c1ba909bbb8ae42e057212b2c6b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 26 Apr 2024 12:58:50 +0300 Subject: [PATCH 56/77] Tmp --- datafusion/core/src/physical_planner.rs | 1 + datafusion/physical-expr-common/src/aggregate/mod.rs | 1 + datafusion/physical-expr/src/equivalence/projection.rs | 10 +++++----- datafusion/sqllogictest/test_files/distinct_on.slt | 5 +++-- 4 files changed, 10 insertions(+), 7 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5fdeba2a9452..c3fdd24820e6 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1991,6 +1991,7 @@ pub fn create_aggregate_expr_and_maybe_filter( Expr::Alias(Alias { expr, name, .. }) => (name.clone(), expr.as_ref()), _ => (physical_name(e)?, e), }; + create_aggregate_expr_with_name_and_maybe_filter( e, name, diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index dc602fab4b52..de806d455813 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -76,6 +76,7 @@ pub fn create_aggregate_expr( })) } +#[allow(dead_code)] fn calc_fn_name_with_args(fn_name: &str, args: &[Arc]) -> String { let args = args .iter() diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 260610f23dc6..ba471c76945e 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{internal_err, Result}; +use datafusion_common::{Result}; use crate::expressions::Column; use crate::PhysicalExpr; @@ -67,10 +67,10 @@ impl ProjectionMapping { // Conceptually, `source_expr` and `expression` should be the same. let idx = col.index(); let matching_input_field = input_schema.field(idx); - if col.name() != matching_input_field.name() { - return internal_err!("Input field name {} does not match with the projection expression {}", - matching_input_field.name(),col.name()) - } + // if col.name() != matching_input_field.name() { + // return internal_err!("Input field name {} does not match with the projection expression {}", + // matching_input_field.name(),col.name()) + // } let matching_input_column = Column::new(matching_input_field.name(), idx); Ok(Transformed::yes(Arc::new(matching_input_column))) diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 932fd42180ad..820d985863aa 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -137,13 +137,14 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] 02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -03)----AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [c3@1 DESC]] +03)----AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[CAST(c2@0 AS Int64) % 2 = 0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(CAST(c3@1 AS Int64) - 100) ORDER BY [c3@1 DESC]] +06)----------AggregateExec: mode=Partial, gby=[CAST(c2@0 AS Int64) % 2 = 0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]] 07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + query BI SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) FROM aggregate_test_100 From 782fa2c498bbb1fcf5dae27103097f1a3ea40188 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 29 Apr 2024 09:33:26 +0300 Subject: [PATCH 57/77] Tmp --- datafusion/physical-expr/src/equivalence/projection.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index ba471c76945e..4dca12a81e74 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::{Result}; +use datafusion_common::Result; use crate::expressions::Column; use crate::PhysicalExpr; From f39e1fbd15204c0d907528ac9b1195a4680a2e99 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 29 Apr 2024 09:55:50 +0300 Subject: [PATCH 58/77] TMP --- datafusion/core/src/physical_planner.rs | 1 + .../src/equivalence/projection.rs | 10 ++--- .../sqllogictest/test_files/distinct_on.slt | 41 +++++++++++++++++-- 3 files changed, 43 insertions(+), 9 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c3fdd24820e6..e76dbe2f224c 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1940,6 +1940,7 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( .iter() .map(|sort_expr| format!("{sort_expr}")) .collect::>(); + // name.into() format!("{name} ORDER BY [{}]", reqs.join(", ")) } } diff --git a/datafusion/physical-expr/src/equivalence/projection.rs b/datafusion/physical-expr/src/equivalence/projection.rs index 4dca12a81e74..260610f23dc6 100644 --- a/datafusion/physical-expr/src/equivalence/projection.rs +++ b/datafusion/physical-expr/src/equivalence/projection.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::Result; +use datafusion_common::{internal_err, Result}; use crate::expressions::Column; use crate::PhysicalExpr; @@ -67,10 +67,10 @@ impl ProjectionMapping { // Conceptually, `source_expr` and `expression` should be the same. let idx = col.index(); let matching_input_field = input_schema.field(idx); - // if col.name() != matching_input_field.name() { - // return internal_err!("Input field name {} does not match with the projection expression {}", - // matching_input_field.name(),col.name()) - // } + if col.name() != matching_input_field.name() { + return internal_err!("Input field name {} does not match with the projection expression {}", + matching_input_field.name(),col.name()) + } let matching_input_column = Column::new(matching_input_field.name(), idx); Ok(Transformed::yes(Arc::new(matching_input_column))) diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 820d985863aa..38426b7a0aca 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -112,10 +112,9 @@ SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 query error DataFusion error: Error during planning: No `ON` expressions provided SELECT DISTINCT ON () c1, c2 FROM aggregate_test_100 ORDER BY c1, c2; -# query BII -# SELECT c2 % 2 = 0, c2, c3 - 100 -# FROM aggregate_test_100 -# ORDER BY c2 % 2 = 0, c3 DESC; +# # Use expressions in the ON and ORDER BY clauses, as well as the selection +# query TT +# EXPLAIN verbose SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; # ---- query II @@ -124,6 +123,40 @@ SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 5 -140 2 -99 +# Use expressions in the ON and ORDER BY clauses, as well as the selection +query TT +EXPLAIN SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +logical_plan +01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) +02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST +03)----Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] +04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, aggregate_test_100.c2, aggregate_test_100.c3 +05)--------TableScan: aggregate_test_100 projection=[c2, c3] +physical_plan +01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] +02)--SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +03)----SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +04)------AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] +05)--------CoalesceBatchesExec: target_batch_size=8192 +06)----------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 +07)------------AggregateExec: mode=Partial, gby=[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] +08)--------------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, c2@0 as c2, c3@1 as c3] +09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true + +query BII +SELECT c2 % 2 = 0, c2, c3 - 100 +FROM aggregate_test_100 +ORDER BY c2 % 2 = 0, c3 DESC; +---- + +query II +SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; +---- +5 -140 +2 -99 + query TT EXPLAIN SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) FROM aggregate_test_100 From 23a48f323398490a0cebb21c127e80d91c5113e9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 9 May 2024 15:50:33 +0300 Subject: [PATCH 59/77] Simplifications --- .../physical_optimizer/update_aggr_exprs.rs | 2 +- .../functions-aggregate/src/first_last.rs | 58 +++++++++---------- datafusion/functions-aggregate/src/lib.rs | 4 +- .../src/replace_distinct_aggregate.rs | 2 +- .../physical-expr/src/aggregate/build_in.rs | 5 +- 5 files changed, 32 insertions(+), 39 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 1e57a2101921..bdebd547e091 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -75,7 +75,7 @@ impl PhysicalOptimizerRule for OptimizeAggregateOrder { let input = aggr_exec.input(); let mut aggr_expr = aggr_exec.aggr_expr().to_vec(); - let groupby_exprs = aggr_exec.group_by().input_exprs(); + let groupby_exprs = aggr_exec.group_expr().input_exprs(); // If the existing ordering satisfies a prefix of the GROUP BY // expressions, prefix requirements with this section. In this // case, aggregation will work more efficiently. diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 52fc94248613..b590bab67a30 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -39,8 +39,6 @@ use datafusion_physical_expr_common::aggregate::utils::get_sort_options; use datafusion_physical_expr_common::expressions; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use sqlparser::ast::NullTreatment; - make_udaf_expr_and_func!( FirstValue, first_value, @@ -87,8 +85,7 @@ impl FirstValue { } fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - // self.requirement_satisfied = requirement_satisfied; - self.requirement_satisfied = false; + self.requirement_satisfied = requirement_satisfied; self } } @@ -749,7 +746,7 @@ impl Accumulator for FirstValueAccumulator { // } // } -make_udaf_function!( +make_udaf_expr_and_func!( LastValue, last_value, "Returns the last value in a group of values.", @@ -795,8 +792,7 @@ impl LastValue { } fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { - // self.requirement_satisfied = requirement_satisfied; - self.requirement_satisfied = false; + self.requirement_satisfied = requirement_satisfied; self } } @@ -1104,30 +1100,30 @@ fn convert_to_sort_cols( .collect::>() } -fn replace_order_by_clause(order_by: &mut String) { - let suffixes = [ - (" DESC NULLS FIRST]", " ASC NULLS LAST]"), - (" ASC NULLS FIRST]", " DESC NULLS LAST]"), - (" DESC NULLS LAST]", " ASC NULLS FIRST]"), - (" ASC NULLS LAST]", " DESC NULLS FIRST]"), - ]; - - if let Some(start) = order_by.find("ORDER BY [") { - if let Some(end) = order_by[start..].find(']') { - let order_by_start = start + 9; - let order_by_end = start + end; - - let column_order = &order_by[order_by_start..=order_by_end]; - for &(suffix, replacement) in &suffixes { - if column_order.ends_with(suffix) { - let new_order = column_order.replace(suffix, replacement); - order_by.replace_range(order_by_start..=order_by_end, &new_order); - break; - } - } - } - } -} +// fn replace_order_by_clause(order_by: &mut String) { +// let suffixes = [ +// (" DESC NULLS FIRST]", " ASC NULLS LAST]"), +// (" ASC NULLS FIRST]", " DESC NULLS LAST]"), +// (" DESC NULLS LAST]", " ASC NULLS FIRST]"), +// (" ASC NULLS LAST]", " DESC NULLS FIRST]"), +// ]; +// +// if let Some(start) = order_by.find("ORDER BY [") { +// if let Some(end) = order_by[start..].find(']') { +// let order_by_start = start + 9; +// let order_by_end = start + end; +// +// let column_order = &order_by[order_by_start..=order_by_end]; +// for &(suffix, replacement) in &suffixes { +// if column_order.ends_with(suffix) { +// let new_order = column_order.replace(suffix, replacement); +// order_by.replace_range(order_by_start..=order_by_end, &new_order); +// break; +// } +// } +// } +// } +// } #[cfg(test)] mod tests { diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 1493aced6808..4bcd9d78320f 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -67,8 +67,8 @@ use std::sync::Arc; /// Fluent-style API for creating `Expr`s pub mod expr_fn { pub use super::covariance::covar_samp; - pub use super::first_last::first_value; - pub use super::first_last::last_value; + // pub use super::first_last::first_value; + // pub use super::first_last::last_value; } /// Registers all enabled packages with a [`FunctionRegistry`] diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index 683962d270ba..6877a8c4b6d2 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -23,7 +23,7 @@ use datafusion_common::{Column, Result}; use datafusion_expr::utils::expand_wildcard; use datafusion_expr::{col, LogicalPlanBuilder}; use datafusion_expr::{Aggregate, Distinct, DistinctOn, Expr, LogicalPlan}; -use datafusion_functions_aggregate::expr_fn::first_value; +use datafusion_functions_aggregate::first_last::first_value; /// Optimizer that replaces logical [[Distinct]] with a logical [[Aggregate]] /// diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 2309fdb363a1..c725698e6450 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -46,7 +46,7 @@ pub fn create_aggregate_expr( ordering_req: &[PhysicalSortExpr], input_schema: &Schema, name: impl Into, - ignore_nulls: bool, + _ignore_nulls: bool, ) -> Result> { let name = name.into(); // get the result data type for this aggregate function @@ -349,9 +349,6 @@ pub fn create_aggregate_expr( data_type, distinct, )), - (AggregateFunction::Median, true) => { - return not_impl_err!("MEDIAN(DISTINCT) aggregations are not available"); - } (AggregateFunction::NthValue, _) => { let expr = &input_phy_exprs[0]; let Some(n) = input_phy_exprs[1] From 0fd29aae0a2b2c8670644de4d561db73eb60a1fb Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 9 May 2024 15:59:20 +0300 Subject: [PATCH 60/77] Tmp --- datafusion/physical-plan/src/aggregates/mod.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index da83939b01e2..bda207628ca2 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1982,6 +1982,7 @@ mod tests { options: SortOptions::default(), }]; let aggregates: Vec> = if is_first_acc { + create_aggregate_expr() vec![Arc::new(FirstValue::new( col("b", &schema)?, "FIRST_VALUE(b)".to_string(), From 78f798ed5fd5bded7152e898f1eccf74d05b5542 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 10 May 2024 09:40:49 +0300 Subject: [PATCH 61/77] Tmp --- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bda207628ca2..29362de0661f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1982,7 +1982,7 @@ mod tests { options: SortOptions::default(), }]; let aggregates: Vec> = if is_first_acc { - create_aggregate_expr() + // create_aggregate_expr() vec![Arc::new(FirstValue::new( col("b", &schema)?, "FIRST_VALUE(b)".to_string(), From 936412972d09c5de706454d81686368d603c478f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 11:29:13 +0300 Subject: [PATCH 62/77] Compiles --- .../functions-aggregate/src/first_last.rs | 400 +----------------- .../physical-expr-common/src/aggregate/mod.rs | 2 + datafusion/physical-expr-common/src/utils.rs | 5 - 3 files changed, 12 insertions(+), 395 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 662e8d5cdf46..00193b4f9395 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -359,388 +359,6 @@ impl Accumulator for FirstValueAccumulator { } } -// /// TO BE DEPRECATED: Builtin FIRST_VALUE physical aggregate expression will be replaced by udf in the future -// #[derive(Debug, Clone)] -// pub struct FirstValuePhysicalExpr { -// name: String, -// input_data_type: DataType, -// order_by_data_types: Vec, -// expr: Arc, -// ordering_req: LexOrdering, -// requirement_satisfied: bool, -// ignore_nulls: bool, -// state_fields: Vec, -// } -// -// impl FirstValuePhysicalExpr { -// /// Creates a new FIRST_VALUE aggregation function. -// pub fn new( -// expr: Arc, -// name: impl Into, -// input_data_type: DataType, -// ordering_req: LexOrdering, -// order_by_data_types: Vec, -// state_fields: Vec, -// ) -> Self { -// let requirement_satisfied = ordering_req.is_empty(); -// Self { -// name: name.into(), -// input_data_type, -// order_by_data_types, -// expr, -// ordering_req, -// requirement_satisfied, -// ignore_nulls: false, -// state_fields, -// } -// } -// -// pub fn with_ignore_nulls(mut self, ignore_nulls: bool) -> Self { -// self.ignore_nulls = ignore_nulls; -// self -// } -// -// /// Returns the name of the aggregate expression. -// pub fn name(&self) -> &str { -// &self.name -// } -// -// /// Returns the input data type of the aggregate expression. -// pub fn input_data_type(&self) -> &DataType { -// &self.input_data_type -// } -// -// /// Returns the data types of the order-by columns. -// pub fn order_by_data_types(&self) -> &Vec { -// &self.order_by_data_types -// } -// -// /// Returns the expression associated with the aggregate function. -// pub fn expr(&self) -> &Arc { -// &self.expr -// } -// -// /// Returns the lexical ordering requirements of the aggregate expression. -// pub fn ordering_req(&self) -> &LexOrdering { -// &self.ordering_req -// } -// -// pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { -// self.requirement_satisfied = requirement_satisfied; -// self -// } -// -// pub fn convert_to_last(self) -> LastValuePhysicalExpr { -// let mut name = format!("LAST{}", &self.name[5..]); -// replace_order_by_clause(&mut name); -// -// let FirstValuePhysicalExpr { -// expr, -// input_data_type, -// ordering_req, -// order_by_data_types, -// .. -// } = self; -// LastValuePhysicalExpr::new( -// expr, -// name, -// input_data_type, -// reverse_order_bys(&ordering_req), -// order_by_data_types, -// ) -// } -// } -// -// impl AggregateExpr for FirstValuePhysicalExpr { -// /// 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.input_data_type.clone(), true)) -// } -// -// fn create_accumulator(&self) -> Result> { -// FirstValueAccumulator::try_new( -// &self.input_data_type, -// &self.order_by_data_types, -// self.ordering_req.clone(), -// self.ignore_nulls, -// ) -// .map(|acc| { -// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ -// }) -// } -// -// fn state_fields(&self) -> Result> { -// if !self.state_fields.is_empty() { -// return Ok(self.state_fields.clone()); -// } -// -// let mut fields = vec![Field::new( -// format_state_name(&self.name, "first_value"), -// self.input_data_type.clone(), -// true, -// )]; -// fields.extend(ordering_fields( -// &self.ordering_req, -// &self.order_by_data_types, -// )); -// fields.push(Field::new( -// format_state_name(&self.name, "is_set"), -// DataType::Boolean, -// true, -// )); -// Ok(fields) -// } -// -// fn expressions(&self) -> Vec> { -// vec![self.expr.clone()] -// } -// -// fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { -// (!self.ordering_req.is_empty()).then_some(&self.ordering_req) -// } -// -// fn order_sensitivity(&self) -> AggregateOrderSensitivity { -// // Can generate correct result, even if the ordering requirement is not satisfied at the input -// // (less efficient compared to ordering satisfied version) -// AggregateOrderSensitivity::Beneficial -// } -// -// fn with_requirement_satisfied( -// self: Arc, -// requirement_satisfied: bool, -// ) -> Result>> { -// let mut new_self = self.as_ref().clone(); -// new_self.requirement_satisfied = requirement_satisfied; -// Ok(Some(Arc::new(new_self))) -// } -// -// fn name(&self) -> &str { -// &self.name -// } -// -// fn reverse_expr(&self) -> Option> { -// Some(Arc::new(self.clone().convert_to_last())) -// } -// -// fn create_sliding_accumulator(&self) -> Result> { -// FirstValueAccumulator::try_new( -// &self.input_data_type, -// &self.order_by_data_types, -// self.ordering_req.clone(), -// self.ignore_nulls, -// ) -// .map(|acc| { -// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ -// }) -// } -// } -// -// impl PartialEq for FirstValuePhysicalExpr { -// fn eq(&self, other: &dyn Any) -> bool { -// down_cast_any_ref(other) -// .downcast_ref::() -// .map(|x| { -// self.name == x.name -// && self.input_data_type == x.input_data_type -// && self.order_by_data_types == x.order_by_data_types -// && self.expr.eq(&x.expr) -// }) -// .unwrap_or(false) -// } -// } -// -// /// TO BE DEPRECATED: Builtin LAST_VALUE physical aggregate expression will be replaced by udf in the future -// #[derive(Debug, Clone)] -// pub struct LastValuePhysicalExpr { -// name: String, -// input_data_type: DataType, -// order_by_data_types: Vec, -// expr: Arc, -// ordering_req: LexOrdering, -// requirement_satisfied: bool, -// ignore_nulls: bool, -// } -// -// impl LastValuePhysicalExpr { -// /// Creates a new LAST_VALUE aggregation function. -// pub fn new( -// expr: Arc, -// name: impl Into, -// input_data_type: DataType, -// ordering_req: LexOrdering, -// order_by_data_types: Vec, -// ) -> Self { -// let requirement_satisfied = ordering_req.is_empty(); -// Self { -// name: name.into(), -// input_data_type, -// order_by_data_types, -// expr, -// ordering_req, -// requirement_satisfied, -// ignore_nulls: false, -// } -// } -// -// pub fn with_ignore_nulls(mut self, ignore_nulls: bool) -> Self { -// self.ignore_nulls = ignore_nulls; -// self -// } -// -// /// Returns the name of the aggregate expression. -// pub fn name(&self) -> &str { -// &self.name -// } -// -// /// Returns the input data type of the aggregate expression. -// pub fn input_data_type(&self) -> &DataType { -// &self.input_data_type -// } -// -// /// Returns the data types of the order-by columns. -// pub fn order_by_data_types(&self) -> &Vec { -// &self.order_by_data_types -// } -// -// /// Returns the expression associated with the aggregate function. -// pub fn expr(&self) -> &Arc { -// &self.expr -// } -// -// /// Returns the lexical ordering requirements of the aggregate expression. -// pub fn ordering_req(&self) -> &LexOrdering { -// &self.ordering_req -// } -// -// pub fn with_requirement_satisfied(mut self, requirement_satisfied: bool) -> Self { -// self.requirement_satisfied = requirement_satisfied; -// self -// } -// -// pub fn convert_to_first(self) -> FirstValuePhysicalExpr { -// let mut name = format!("FIRST{}", &self.name[4..]); -// replace_order_by_clause(&mut name); -// -// let LastValuePhysicalExpr { -// expr, -// input_data_type, -// ordering_req, -// order_by_data_types, -// .. -// } = self; -// FirstValuePhysicalExpr::new( -// expr, -// name, -// input_data_type, -// reverse_order_bys(&ordering_req), -// order_by_data_types, -// vec![], -// ) -// } -// } -// -// impl AggregateExpr for LastValuePhysicalExpr { -// /// 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.input_data_type.clone(), true)) -// } -// -// fn create_accumulator(&self) -> Result> { -// LastValueAccumulator::try_new( -// &self.input_data_type, -// &self.order_by_data_types, -// self.ordering_req.clone(), -// self.ignore_nulls, -// ) -// .map(|acc| { -// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ -// }) -// } -// -// fn state_fields(&self) -> Result> { -// let mut fields = vec![Field::new( -// format_state_name(&self.name, "last_value"), -// self.input_data_type.clone(), -// true, -// )]; -// fields.extend(ordering_fields( -// &self.ordering_req, -// &self.order_by_data_types, -// )); -// fields.push(Field::new( -// format_state_name(&self.name, "is_set"), -// DataType::Boolean, -// true, -// )); -// Ok(fields) -// } -// -// fn expressions(&self) -> Vec> { -// vec![self.expr.clone()] -// } -// -// fn order_bys(&self) -> Option<&[PhysicalSortExpr]> { -// (!self.ordering_req.is_empty()).then_some(&self.ordering_req) -// } -// -// fn order_sensitivity(&self) -> AggregateOrderSensitivity { -// // Can generate correct result, even if the ordering requirement is not satisfied at the input -// // (less efficient compared to ordering satisfied version) -// AggregateOrderSensitivity::Beneficial -// } -// -// fn with_requirement_satisfied( -// self: Arc, -// requirement_satisfied: bool, -// ) -> Result>> { -// let mut new_self = self.as_ref().clone(); -// new_self.requirement_satisfied = requirement_satisfied; -// Ok(Some(Arc::new(new_self))) -// } -// -// fn name(&self) -> &str { -// &self.name -// } -// -// fn reverse_expr(&self) -> Option> { -// Some(Arc::new(self.clone().convert_to_first())) -// } -// -// fn create_sliding_accumulator(&self) -> Result> { -// LastValueAccumulator::try_new( -// &self.input_data_type, -// &self.order_by_data_types, -// self.ordering_req.clone(), -// self.ignore_nulls, -// ) -// .map(|acc| { -// Box::new(acc.with_requirement_satisfied(self.requirement_satisfied)) as _ -// }) -// } -// } -// -// impl PartialEq for LastValuePhysicalExpr { -// fn eq(&self, other: &dyn Any) -> bool { -// down_cast_any_ref(other) -// .downcast_ref::() -// .map(|x| { -// self.name == x.name -// && self.input_data_type == x.input_data_type -// && self.order_by_data_types == x.order_by_data_types -// && self.expr.eq(&x.expr) -// }) -// .unwrap_or(false) -// } -// } - make_udaf_expr_and_func!( LastValue, last_value, @@ -851,18 +469,20 @@ impl AggregateUDFImpl for LastValue { .map(|acc| Box::new(acc.with_requirement_satisfied(requirement_satisfied)) as _) } - fn state_fields( - &self, - name: &str, - value_type: DataType, - ordering_fields: Vec, - ) -> Result> { + fn state_fields(&self, args: StateFieldsArgs) -> Result> { + let StateFieldsArgs { + name, + input_type: _, + return_type, + ordering_fields, + is_distinct: _, + } = args; let mut fields = vec![Field::new( format_state_name(name, "last_value"), - value_type, + return_type.clone(), true, )]; - fields.extend(ordering_fields); + fields.extend(ordering_fields.to_vec()); fields.push(Field::new("is_set", DataType::Boolean, true)); Ok(fields) } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 2d9abd0b2197..f49a8fdfabd1 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -373,6 +373,7 @@ impl AggregateExpr for AggregateFunctionExpr { &self.schema, self.name(), self.ignore_nulls, + self.is_distinct, ) .unwrap(); return Ok(Some(aggr_expr)); @@ -397,6 +398,7 @@ impl AggregateExpr for AggregateFunctionExpr { &self.schema, name, self.ignore_nulls, + self.is_distinct, ) .unwrap(); return Some(reverse_aggr); diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 2113a5ef7c89..c0ee727da5d3 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -17,11 +17,6 @@ use std::sync::Arc; -use arrow::{ - array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}, - compute::{and_kleene, is_not_null, SlicesIterator}, -}; -use datafusion_common::Result; use datafusion_expr::Expr; use crate::{ From 2816c5b9f40e649c9e865797c4ad6665c2b2c7a6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 13:09:07 +0300 Subject: [PATCH 63/77] Resolve linter errors --- datafusion/core/src/physical_planner.rs | 14 +- .../physical-plan/src/aggregates/mod.rs | 98 +++++++++----- datafusion/proto/proto/datafusion.proto | 8 +- datafusion/proto/src/generated/pbjson.rs | 6 - datafusion/proto/src/generated/prost.rs | 12 +- .../proto/src/logical_plan/from_proto.rs | 2 - datafusion/proto/src/logical_plan/to_proto.rs | 8 -- .../proto/src/physical_plan/to_proto.rs | 13 +- .../tests/cases/roundtrip_logical_plan.rs | 2 +- .../sqllogictest/test_files/distinct_on.slt | 120 ++++++++++++++++-- 10 files changed, 196 insertions(+), 87 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 5b1b85efe990..f659699ec184 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1881,13 +1881,13 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( if order_by.is_empty() { name.into() } else { - let name = name.into(); - let reqs = order_by - .iter() - .map(|sort_expr| format!("{sort_expr}")) - .collect::>(); - // name.into() - format!("{name} ORDER BY [{}]", reqs.join(", ")) + // let name = name.into(); + // let reqs = order_by + // .iter() + // .map(|sort_expr| format!("{sort_expr}")) + // .collect::>(); + // format!("{name} ORDER BY [{}]", reqs.join(", ")) + name.into() } } None => name.into(), diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3e13a2ac0e3c..1eaad3e3314d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1198,6 +1198,7 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; + use datafusion_expr::expr::Sort; use datafusion_physical_expr::expressions::{ lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, }; @@ -1977,28 +1978,52 @@ mod tests { let groups = PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); + let sort_options = SortOptions { + descending: false, + nulls_first: false, + }; + let sort_exprs = vec![datafusion_expr::Expr::Sort(Sort { + expr: Box::new(datafusion_expr::Expr::Column( + datafusion_common::Column::new(Some("table1"), "b"), + )), + asc: !sort_options.descending, + nulls_first: sort_options.nulls_first, + })]; let ordering_req = vec![PhysicalSortExpr { expr: col("b", &schema)?, - options: SortOptions::default(), + options: sort_options, }]; + let args = vec![col("b", &schema)?]; let aggregates: Vec> = if is_first_acc { - // create_aggregate_expr() - vec![Arc::new(FirstValue::new( - col("b", &schema)?, - "FIRST_VALUE(b)".to_string(), - DataType::Float64, - ordering_req.clone(), - vec![DataType::Float64], - vec![], - ))] + let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); + vec![ + datafusion_physical_expr_common::aggregate::create_aggregate_expr( + &func, + &args, + &sort_exprs, + &ordering_req, + &schema, + "FIRST_VALUE(b)", + false, + false, + ) + .unwrap(), + ] } else { - vec![Arc::new(LastValue::new( - col("b", &schema)?, - "LAST_VALUE(b)".to_string(), - DataType::Float64, - ordering_req.clone(), - vec![DataType::Float64], - ))] + let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); + vec![ + datafusion_physical_expr_common::aggregate::create_aggregate_expr( + &func, + &args, + &sort_exprs, + &ordering_req, + &schema, + "LAST_VALUE(b)", + false, + false, + ) + .unwrap(), + ] }; let memory_exec = Arc::new(MemoryExec::try_new( @@ -2165,22 +2190,31 @@ mod tests { let sort_expr_reverse = reverse_order_bys(&sort_expr); let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); + let first_func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); + let last_func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); let aggregates: Vec> = vec![ - Arc::new(FirstValue::new( - col_b.clone(), - "FIRST_VALUE(b)".to_string(), - DataType::Float64, - sort_expr_reverse.clone(), - vec![DataType::Float64], - vec![], - )), - Arc::new(LastValue::new( - col_b.clone(), - "LAST_VALUE(b)".to_string(), - DataType::Float64, - sort_expr.clone(), - vec![DataType::Float64], - )), + datafusion_physical_expr_common::aggregate::create_aggregate_expr( + &first_func, + &[col_b.clone()], + &[], + &sort_expr_reverse, + &schema, + "FIRST_VALUE(b)", + false, + false, + ) + .unwrap(), + datafusion_physical_expr_common::aggregate::create_aggregate_expr( + &last_func, + &[col_b.clone()], + &[], + &sort_expr, + &schema, + "LAST_VALUE(b)", + false, + false, + ) + .unwrap(), ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let aggregate_exec = Arc::new(AggregateExec::try_new( diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 73e751c616ac..d2c0a42b2f37 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -557,10 +557,10 @@ enum AggregateFunction { BIT_XOR = 21; BOOL_AND = 22; BOOL_OR = 23; - // 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; +// // 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; REGR_SLOPE = 26; REGR_INTERCEPT = 27; REGR_COUNT = 28; diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 77ba0808fb77..c00098d19385 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -443,8 +443,6 @@ impl serde::Serialize for AggregateFunction { Self::BitXor => "BIT_XOR", Self::BoolAnd => "BOOL_AND", Self::BoolOr => "BOOL_OR", - Self::FirstValueAgg => "FIRST_VALUE_AGG", - Self::LastValueAgg => "LAST_VALUE_AGG", Self::RegrSlope => "REGR_SLOPE", Self::RegrIntercept => "REGR_INTERCEPT", Self::RegrCount => "REGR_COUNT", @@ -489,8 +487,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "BIT_XOR", "BOOL_AND", "BOOL_OR", - "FIRST_VALUE_AGG", - "LAST_VALUE_AGG", "REGR_SLOPE", "REGR_INTERCEPT", "REGR_COUNT", @@ -564,8 +560,6 @@ impl<'de> serde::Deserialize<'de> for AggregateFunction { "BIT_XOR" => Ok(AggregateFunction::BitXor), "BOOL_AND" => Ok(AggregateFunction::BoolAnd), "BOOL_OR" => Ok(AggregateFunction::BoolOr), - "FIRST_VALUE_AGG" => Ok(AggregateFunction::FirstValueAgg), - "LAST_VALUE_AGG" => Ok(AggregateFunction::LastValueAgg), "REGR_SLOPE" => Ok(AggregateFunction::RegrSlope), "REGR_INTERCEPT" => Ok(AggregateFunction::RegrIntercept), "REGR_COUNT" => Ok(AggregateFunction::RegrCount), diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index a175987f1994..dff7088434b3 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2854,10 +2854,10 @@ pub enum AggregateFunction { BitXor = 21, BoolAnd = 22, BoolOr = 23, - /// 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, + /// // 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; RegrSlope = 26, RegrIntercept = 27, RegrCount = 28, @@ -2901,8 +2901,6 @@ impl AggregateFunction { AggregateFunction::BitXor => "BIT_XOR", AggregateFunction::BoolAnd => "BOOL_AND", AggregateFunction::BoolOr => "BOOL_OR", - AggregateFunction::FirstValueAgg => "FIRST_VALUE_AGG", - AggregateFunction::LastValueAgg => "LAST_VALUE_AGG", AggregateFunction::RegrSlope => "REGR_SLOPE", AggregateFunction::RegrIntercept => "REGR_INTERCEPT", AggregateFunction::RegrCount => "REGR_COUNT", @@ -2943,8 +2941,6 @@ impl AggregateFunction { "BIT_XOR" => Some(Self::BitXor), "BOOL_AND" => Some(Self::BoolAnd), "BOOL_OR" => Some(Self::BoolOr), - "FIRST_VALUE_AGG" => Some(Self::FirstValueAgg), - "LAST_VALUE_AGG" => Some(Self::LastValueAgg), "REGR_SLOPE" => Some(Self::RegrSlope), "REGR_INTERCEPT" => Some(Self::RegrIntercept), "REGR_COUNT" => Some(Self::RegrCount), diff --git a/datafusion/proto/src/logical_plan/from_proto.rs b/datafusion/proto/src/logical_plan/from_proto.rs index b6f72f6773a2..1f9322b074c5 100644 --- a/datafusion/proto/src/logical_plan/from_proto.rs +++ b/datafusion/proto/src/logical_plan/from_proto.rs @@ -451,8 +451,6 @@ impl From for AggregateFunction { protobuf::AggregateFunction::ApproxMedian => Self::ApproxMedian, protobuf::AggregateFunction::Grouping => Self::Grouping, protobuf::AggregateFunction::Median => Self::Median, - protobuf::AggregateFunction::FirstValueAgg => Self::FirstValue, - protobuf::AggregateFunction::LastValueAgg => Self::LastValue, protobuf::AggregateFunction::NthValueAgg => Self::NthValue, protobuf::AggregateFunction::StringAgg => Self::StringAgg, } diff --git a/datafusion/proto/src/logical_plan/to_proto.rs b/datafusion/proto/src/logical_plan/to_proto.rs index 91f7411e911a..a2d805a92c9e 100644 --- a/datafusion/proto/src/logical_plan/to_proto.rs +++ b/datafusion/proto/src/logical_plan/to_proto.rs @@ -387,8 +387,6 @@ impl From<&AggregateFunction> for protobuf::AggregateFunction { AggregateFunction::ApproxMedian => Self::ApproxMedian, AggregateFunction::Grouping => Self::Grouping, AggregateFunction::Median => Self::Median, - AggregateFunction::FirstValue => Self::FirstValueAgg, - AggregateFunction::LastValue => Self::LastValueAgg, AggregateFunction::NthValue => Self::NthValueAgg, AggregateFunction::StringAgg => Self::StringAgg, } @@ -698,12 +696,6 @@ pub fn serialize_expr( } AggregateFunction::Grouping => protobuf::AggregateFunction::Grouping, AggregateFunction::Median => protobuf::AggregateFunction::Median, - AggregateFunction::FirstValue => { - protobuf::AggregateFunction::FirstValueAgg - } - AggregateFunction::LastValue => { - protobuf::AggregateFunction::LastValueAgg - } AggregateFunction::NthValue => { protobuf::AggregateFunction::NthValueAgg } diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index c6b94a934f23..d62eb16c878f 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -26,11 +26,10 @@ use datafusion::physical_plan::expressions::{ ApproxDistinct, ApproxMedian, ApproxPercentileCont, ApproxPercentileContWithWeight, ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, CastExpr, Column, Correlation, Count, CumeDist, DistinctArrayAgg, DistinctBitXor, - DistinctCount, DistinctSum, FirstValue, Grouping, InListExpr, IsNotNullExpr, - IsNullExpr, LastValue, Literal, Max, Median, Min, NegativeExpr, NotExpr, NthValue, - NthValueAgg, Ntile, OrderSensitiveArrayAgg, Rank, RankType, Regr, RegrType, - RowNumber, Stddev, StddevPop, StringAgg, Sum, TryCastExpr, Variance, VariancePop, - WindowShift, + DistinctCount, DistinctSum, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, + Max, Median, Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, + OrderSensitiveArrayAgg, Rank, RankType, Regr, RegrType, RowNumber, Stddev, StddevPop, + StringAgg, Sum, TryCastExpr, Variance, VariancePop, WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; @@ -320,10 +319,6 @@ fn aggr_expr_to_aggr_fn(expr: &dyn AggregateExpr) -> Result { protobuf::AggregateFunction::ApproxMedian } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::Median - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::FirstValueAgg - } else if aggr_expr.downcast_ref::().is_some() { - protobuf::AggregateFunction::LastValueAgg } else if aggr_expr.downcast_ref::().is_some() { protobuf::AggregateFunction::StringAgg } else if aggr_expr.downcast_ref::().is_some() { diff --git a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs index 6e819ef5bf46..c1d63b2eb435 100644 --- a/datafusion/proto/tests/cases/roundtrip_logical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_logical_plan.rs @@ -32,7 +32,7 @@ use datafusion::execution::context::SessionState; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::execution::FunctionRegistry; use datafusion::functions_aggregate::covariance::{covar_pop, covar_samp}; -use datafusion::functions_aggregate::expr_fn::first_value; +use datafusion::functions_aggregate::first_last::first_value; use datafusion::prelude::*; use datafusion::test_util::{TestTableFactory, TestTableProvider}; use datafusion_common::config::{FormatOptions, TableOptions}; diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 521ef3f67971..95e06dda23c8 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -130,18 +130,18 @@ EXPLAIN SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORD logical_plan 01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) 02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST -03)----Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] -04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, aggregate_test_100.c2, aggregate_test_100.c3 +03)----Aggregate: groupBy=[[{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}} AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}} AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}} AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] +04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS {CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}}, aggregate_test_100.c2, aggregate_test_100.c3 05)--------TableScan: aggregate_test_100 projection=[c2, c3] physical_plan 01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] 02)--SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -03)----SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -04)------AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] +03)----SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST], preserve_partitioning=[true] +04)------AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2 AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] -08)--------------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)Int64(0)CAST(aggregate_test_100.c2 AS Int64) % Int64(2)Int64(2)CAST(aggregate_test_100.c2 AS Int64)aggregate_test_100.c2, c2@0 as c2, c3@1 as c3] +07)------------AggregateExec: mode=Partial, gby=[{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}}@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] +08)--------------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as {CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}}, c2@0 as c2, c3@1 as c3] 09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true @@ -150,6 +150,106 @@ SELECT c2 % 2 = 0, c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ---- +false 1 25 +false 3 23 +false 1 20 +false 5 18 +false 3 12 +false 3 4 +false 1 3 +false 3 -3 +false 1 -17 +false 3 -23 +false 3 -27 +false 1 -29 +false 3 -29 +false 1 -30 +false 5 -32 +false 5 -36 +false 5 -38 +false 1 -43 +false 1 -46 +false 1 -59 +false 1 -62 +false 5 -64 +false 1 -64 +false 1 -71 +false 3 -78 +false 3 -83 +false 3 -83 +false 3 -86 +false 3 -87 +false 3 -87 +false 1 -88 +false 3 -102 +false 5 -105 +false 1 -105 +false 1 -108 +false 3 -112 +false 1 -124 +false 1 -125 +false 5 -131 +false 5 -140 +false 5 -144 +false 1 -156 +false 5 -159 +false 3 -172 +false 1 -172 +false 3 -176 +false 5 -182 +false 1 -185 +false 5 -186 +false 5 -194 +false 3 -195 +false 1 -198 +false 1 -199 +false 5 -201 +false 3 -201 +true 4 23 +true 2 22 +true 2 13 +true 4 2 +true 4 -3 +true 2 -3 +true 4 -4 +true 2 -7 +true 4 -26 +true 4 -27 +true 2 -32 +true 4 -35 +true 2 -37 +true 4 -45 +true 2 -48 +true 2 -48 +true 2 -51 +true 4 -53 +true 2 -55 +true 2 -69 +true 4 -70 +true 2 -71 +true 4 -83 +true 4 -95 +true 4 -97 +true 2 -99 +true 2 -129 +true 4 -138 +true 2 -143 +true 2 -148 +true 4 -153 +true 4 -154 +true 4 -156 +true 4 -159 +true 2 -160 +true 2 -160 +true 2 -161 +true 4 -179 +true 4 -190 +true 4 -201 +true 2 -206 +true 2 -207 +true 4 -211 +true 2 -217 +true 4 -217 query II SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; @@ -169,7 +269,7 @@ logical_plan 03)----TableScan: aggregate_test_100 projection=[c2, c3] physical_plan 01)SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] +02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 @@ -197,8 +297,8 @@ SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 query II SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ---- -1 25 -4 23 +5 -140 +2 -99 # Multiple complex expressions query TIB @@ -208,7 +308,7 @@ WHERE c1 IN ('a', 'b') ORDER BY chr(ascii(c1) + 3), c2 % 2, c3 DESC; ---- D 0 false -D 1 true +D 1 false E 0 false E 1 false From 910f0ecf3e30978324000691428266dbbf886b31 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 13:09:16 +0300 Subject: [PATCH 64/77] Resolve linter errors --- datafusion-cli/Cargo.lock | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 99af80bf9df2..f9562b351670 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1323,6 +1323,7 @@ dependencies = [ "chrono", "datafusion-common", "datafusion-expr", + "datafusion-functions-aggregate", "datafusion-physical-expr", "hashbrown 0.14.5", "indexmap 2.2.6", From ba403dc81eae634d9e2132e614b8d006f6974c7f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 13:16:57 +0300 Subject: [PATCH 65/77] Minor changes --- datafusion/functions-aggregate/src/lib.rs | 2 - .../sqllogictest/test_files/distinct_on.slt | 181 ------------------ 2 files changed, 183 deletions(-) diff --git a/datafusion/functions-aggregate/src/lib.rs b/datafusion/functions-aggregate/src/lib.rs index 5e0ec6cecc0f..7ab5c8fcb856 100644 --- a/datafusion/functions-aggregate/src/lib.rs +++ b/datafusion/functions-aggregate/src/lib.rs @@ -67,8 +67,6 @@ use std::sync::Arc; /// Fluent-style API for creating `Expr`s pub mod expr_fn { pub use super::covariance::covar_samp; - // pub use super::first_last::first_value; - // pub use super::first_last::last_value; } /// Registers all enabled packages with a [`FunctionRegistry`] diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 95e06dda23c8..97b988914d82 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -112,187 +112,6 @@ SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 query error DataFusion error: Error during planning: No `ON` expressions provided SELECT DISTINCT ON () c1, c2 FROM aggregate_test_100 ORDER BY c1, c2; -# # Use expressions in the ON and ORDER BY clauses, as well as the selection -# query TT -# EXPLAIN verbose SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; -# ---- - -query II -SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ----- -5 -140 -2 -99 - -# Use expressions in the ON and ORDER BY clauses, as well as the selection -query TT -EXPLAIN SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ----- -logical_plan -01)Projection: FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS aggregate_test_100.c3 - Int64(100) -02)--Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST -03)----Aggregate: groupBy=[[{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}} AS aggregate_test_100.c2 % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}} AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}} AS aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST] AS FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]]] -04)------Projection: CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) AS {CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}}, aggregate_test_100.c2, aggregate_test_100.c3 -05)--------TableScan: aggregate_test_100 projection=[c2, c3] -physical_plan -01)ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@1 as c2, FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]@2 as aggregate_test_100.c3 - Int64(100)] -02)--SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -03)----SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST], preserve_partitioning=[true] -04)------AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] -05)--------CoalesceBatchesExec: target_batch_size=8192 -06)----------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 -07)------------AggregateExec: mode=Partial, gby=[{CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}}@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c2) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST], FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0) ASC NULLS LAST, aggregate_test_100.c3 DESC NULLS FIRST]] -08)--------------ProjectionExec: expr=[CAST(c2@0 AS Int64) % 2 = 0 as {CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)|{Int64(0)}|{CAST(aggregate_test_100.c2 AS Int64) % Int64(2)|{Int64(2)}|{CAST(aggregate_test_100.c2 AS Int64)|{aggregate_test_100.c2}}}}, c2@0 as c2, c3@1 as c3] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - -query BII -SELECT c2 % 2 = 0, c2, c3 - 100 -FROM aggregate_test_100 -ORDER BY c2 % 2 = 0, c3 DESC; ----- -false 1 25 -false 3 23 -false 1 20 -false 5 18 -false 3 12 -false 3 4 -false 1 3 -false 3 -3 -false 1 -17 -false 3 -23 -false 3 -27 -false 1 -29 -false 3 -29 -false 1 -30 -false 5 -32 -false 5 -36 -false 5 -38 -false 1 -43 -false 1 -46 -false 1 -59 -false 1 -62 -false 5 -64 -false 1 -64 -false 1 -71 -false 3 -78 -false 3 -83 -false 3 -83 -false 3 -86 -false 3 -87 -false 3 -87 -false 1 -88 -false 3 -102 -false 5 -105 -false 1 -105 -false 1 -108 -false 3 -112 -false 1 -124 -false 1 -125 -false 5 -131 -false 5 -140 -false 5 -144 -false 1 -156 -false 5 -159 -false 3 -172 -false 1 -172 -false 3 -176 -false 5 -182 -false 1 -185 -false 5 -186 -false 5 -194 -false 3 -195 -false 1 -198 -false 1 -199 -false 5 -201 -false 3 -201 -true 4 23 -true 2 22 -true 2 13 -true 4 2 -true 4 -3 -true 2 -3 -true 4 -4 -true 2 -7 -true 4 -26 -true 4 -27 -true 2 -32 -true 4 -35 -true 2 -37 -true 4 -45 -true 2 -48 -true 2 -48 -true 2 -51 -true 4 -53 -true 2 -55 -true 2 -69 -true 4 -70 -true 2 -71 -true 4 -83 -true 4 -95 -true 4 -97 -true 2 -99 -true 2 -129 -true 4 -138 -true 2 -143 -true 2 -148 -true 4 -153 -true 4 -154 -true 4 -156 -true 4 -159 -true 2 -160 -true 2 -160 -true 2 -161 -true 4 -179 -true 4 -190 -true 4 -201 -true 2 -206 -true 2 -207 -true 4 -211 -true 2 -217 -true 4 -217 - -query II -SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ----- -5 -140 -2 -99 - -query TT -EXPLAIN SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) -FROM aggregate_test_100 -GROUP BY c2 % 2 = 0 -ORDER BY c2 % 2 = 0; ----- -logical_plan -01)Sort: aggregate_test_100.c2 % Int64(2) = Int64(0) ASC NULLS LAST -02)--Aggregate: groupBy=[[CAST(aggregate_test_100.c2 AS Int64) % Int64(2) = Int64(0)]], aggr=[[FIRST_VALUE(CAST(aggregate_test_100.c3 AS Int64) - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]]] -03)----TableScan: aggregate_test_100 projection=[c2, c3] -physical_plan -01)SortPreservingMergeExec: [aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST] -02)--SortExec: expr=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 ASC NULLS LAST], preserve_partitioning=[true] -03)----AggregateExec: mode=FinalPartitioned, gby=[aggregate_test_100.c2 % Int64(2) = Int64(0)@0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]] -04)------CoalesceBatchesExec: target_batch_size=8192 -05)--------RepartitionExec: partitioning=Hash([aggregate_test_100.c2 % Int64(2) = Int64(0)@0], 4), input_partitions=4 -06)----------AggregateExec: mode=Partial, gby=[CAST(c2@0 AS Int64) % 2 = 0 as aggregate_test_100.c2 % Int64(2) = Int64(0)], aggr=[FIRST_VALUE(aggregate_test_100.c3 - Int64(100)) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST]] -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true - - -query BI -SELECT c2 % 2 = 0, FIRST_VALUE(c3-100 ORDER BY c3 DESC) -FROM aggregate_test_100 -GROUP BY c2 % 2 = 0 -ORDER BY c2 % 2 = 0; ----- -false 25 -true 23 - -query II -SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ----- -5 -140 -2 -99 - # Use expressions in the ON and ORDER BY clauses, as well as the selection query II SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; From 1da66f59d121730b8a3a35439749c6efa551ac99 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 13:21:23 +0300 Subject: [PATCH 66/77] Simplifications --- datafusion/core/src/physical_planner.rs | 16 ---------------- .../physical-expr-common/src/aggregate/mod.rs | 13 +------------ datafusion/proto/proto/datafusion.proto | 4 ---- 3 files changed, 1 insertion(+), 32 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index f659699ec184..bc5818361b7d 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -1876,22 +1876,6 @@ pub fn create_aggregate_expr_with_name_and_maybe_filter( (agg_expr, filter, physical_sort_exprs) } AggregateFunctionDefinition::UDF(fun) => { - let name = match order_by { - Some(order_by) => { - if order_by.is_empty() { - name.into() - } else { - // let name = name.into(); - // let reqs = order_by - // .iter() - // .map(|sort_expr| format!("{sort_expr}")) - // .collect::>(); - // format!("{name} ORDER BY [{}]", reqs.join(", ")) - name.into() - } - } - None => name.into(), - }; let sort_exprs = order_by.clone().unwrap_or(vec![]); let physical_sort_exprs = match order_by { Some(exprs) => Some(create_physical_sort_exprs( diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index f49a8fdfabd1..611535d2ee36 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -66,13 +66,12 @@ pub fn create_aggregate_expr( .collect::>>()?; let ordering_fields = ordering_fields(ordering_req, &ordering_types); - let name = name.into(); Ok(Arc::new(AggregateFunctionExpr { fun: fun.clone(), args: input_phy_exprs.to_vec(), data_type: fun.return_type(&input_exprs_types)?, - name, + name: name.into(), schema: schema.clone(), sort_exprs: sort_exprs.to_vec(), ordering_req: ordering_req.to_vec(), @@ -83,15 +82,6 @@ pub fn create_aggregate_expr( })) } -#[allow(dead_code)] -fn calc_fn_name_with_args(fn_name: &str, args: &[Arc]) -> String { - let args = args - .iter() - .map(|sort_expr| format!("{sort_expr}")) - .collect::>(); - format!("{fn_name}({})", args.join(", ")) -} - /// An aggregate expression that: /// * knows its resulting field /// * knows how to create its accumulator @@ -364,7 +354,6 @@ impl AggregateExpr for AggregateFunctionExpr { .clone() .with_requirement_satisfied(requirement_satisfied)? { - // let name = calc_fn_name_with_args(self.fun.name(), &self.args); let aggr_expr = create_aggregate_expr( &updated_fn, &self.args, diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index d2c0a42b2f37..aa1030f5ba8c 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -557,10 +557,6 @@ enum AggregateFunction { BIT_XOR = 21; BOOL_AND = 22; BOOL_OR = 23; -// // 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; REGR_SLOPE = 26; REGR_INTERCEPT = 27; REGR_COUNT = 28; From 5fa6b5a42200166838bbcabca5331736bbd2e3ab Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 13:33:20 +0300 Subject: [PATCH 67/77] Minor chagnes --- datafusion/expr/src/lib.rs | 1 + datafusion/expr/src/udaf.rs | 11 +++++------ datafusion/functions-aggregate/src/first_last.rs | 8 ++++---- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index bac2f9c14541..f343f28fd3f5 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -81,6 +81,7 @@ pub use signature::{ ArrayFunctionSignature, Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD, }; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; +pub use udaf::ReversedUDAF; pub use udaf::{AggregateUDF, AggregateUDFImpl}; pub use udf::{ScalarUDF, ScalarUDFImpl}; pub use udwf::{WindowUDF, WindowUDFImpl}; diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index d264fc679133..033d368a1f0f 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -209,9 +209,11 @@ impl AggregateUDF { } pub fn reverse_udf(&self) -> Option { - self.inner - .reverse_udf() - .map(|reverse| Self { inner: reverse }) + match self.inner.reverse_expr() { + ReversedUDAF::NotSupported => None, + ReversedUDAF::Identical => Some(self.clone()), + ReversedUDAF::Reversed(reverse) => Some(Self { inner: reverse }), + } } pub fn coerce_types(&self, _args: &[DataType]) -> Result> { @@ -394,9 +396,6 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { AggregateOrderSensitivity::HardRequirement } - fn reverse_udf(&self) -> Option> { - None - } /// Optionally apply per-UDaF simplification / rewrite rules. /// /// This can be used to apply function specific simplification rules during diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 00193b4f9395..ced4d626a38e 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -179,8 +179,8 @@ impl AggregateUDFImpl for FirstValue { AggregateOrderSensitivity::Beneficial } - fn reverse_udf(&self) -> Option> { - Some(Arc::new(LastValue::new())) + fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { + datafusion_expr::ReversedUDAF::Reversed(Arc::new(LastValue::new())) } } @@ -504,8 +504,8 @@ impl AggregateUDFImpl for LastValue { AggregateOrderSensitivity::Beneficial } - fn reverse_udf(&self) -> Option> { - Some(Arc::new(FirstValue::new())) + fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { + datafusion_expr::ReversedUDAF::Reversed(Arc::new(FirstValue::new())) } } From 2aa4244a2bd8fbf82ca7ce1d55643dd8e51641c0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 15:12:18 +0300 Subject: [PATCH 68/77] Move cast to common --- datafusion-cli/Cargo.lock | 1 + datafusion/core/Cargo.toml | 1 + .../core/src/physical_optimizer/pruning.rs | 7 +- .../functions-aggregate/src/first_last.rs | 94 +++---------------- .../physical-expr-common/src/aggregate/mod.rs | 1 + .../src/expressions/cast.rs | 6 +- .../src/expressions/mod.rs | 3 + .../physical-expr-common/src/sort_expr.rs | 27 +++++- datafusion/physical-expr-common/src/utils.rs | 29 +++++- .../src/equivalence/properties.rs | 3 +- .../physical-expr/src/expressions/mod.rs | 3 +- .../physical-expr/src/intervals/utils.rs | 3 +- datafusion/physical-expr/src/planner.rs | 3 +- datafusion/physical-plan/src/lib.rs | 1 + .../proto/src/physical_plan/from_proto.rs | 6 +- .../proto/src/physical_plan/to_proto.rs | 14 +-- .../tests/cases/roundtrip_physical_plan.rs | 5 +- 17 files changed, 100 insertions(+), 107 deletions(-) rename datafusion/{physical-expr => physical-expr-common}/src/expressions/cast.rs (99%) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index f9562b351670..5f146e238492 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1140,6 +1140,7 @@ dependencies = [ "datafusion-functions-array", "datafusion-optimizer", "datafusion-physical-expr", + "datafusion-physical-expr-common", "datafusion-physical-plan", "datafusion-sql", "flate2", diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index a88cdfa27432..f79024ff0a55 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -105,6 +105,7 @@ datafusion-functions-aggregate = { workspace = true } datafusion-functions-array = { workspace = true, optional = true } datafusion-optimizer = { workspace = true } datafusion-physical-expr = { workspace = true } +datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } datafusion-sql = { workspace = true } flate2 = { version = "1.0.24", optional = true } diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 605ef9f9023f..605d38896427 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -43,6 +43,7 @@ use datafusion_common::{ }; use datafusion_physical_expr::utils::{collect_columns, Guarantee, LiteralGuarantee}; use datafusion_physical_expr::{expressions as phys_expr, PhysicalExprRef}; +use datafusion_physical_expr_common::expressions as phys_expr_common; use log::trace; @@ -1076,14 +1077,16 @@ fn rewrite_expr_to_prunable( { // `col op lit()` Ok((column_expr.clone(), op, scalar_expr.clone())) - } else if let Some(cast) = column_expr_any.downcast_ref::() { + } else if let Some(cast) = + column_expr_any.downcast_ref::() + { // `cast(col) op lit()` let arrow_schema: SchemaRef = schema.clone().into(); let from_type = cast.expr().data_type(&arrow_schema)?; verify_support_type_for_prune(&from_type, cast.cast_type())?; let (left, op, right) = rewrite_expr_to_prunable(cast.expr(), op, scalar_expr, schema)?; - let left = Arc::new(phys_expr::CastExpr::new( + let left = Arc::new(phys_expr_common::CastExpr::new( left, cast.cast_type().clone(), None, diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index ced4d626a38e..204713282228 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -22,7 +22,7 @@ use std::fmt::Debug; use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray}; -use arrow::compute::{self, lexsort_to_indices, SortColumn, SortOptions}; +use arrow::compute::{self, lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field}; use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; use datafusion_common::{ @@ -32,12 +32,13 @@ use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::type_coercion::aggregates::NUMERICS; use datafusion_expr::utils::{format_state_name, AggregateOrderSensitivity}; use datafusion_expr::{ - Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Expr, Signature, - TypeSignature, Volatility, + Accumulator, AggregateUDFImpl, ArrayFunctionSignature, Signature, TypeSignature, + Volatility, }; use datafusion_physical_expr_common::aggregate::utils::get_sort_options; -use datafusion_physical_expr_common::expressions; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::{ + convert_logical_sort_exprs_to_physical, LexOrdering, PhysicalSortExpr, +}; make_udaf_expr_and_func!( FirstValue, @@ -108,30 +109,8 @@ impl AggregateUDFImpl for FirstValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let mut all_sort_orders = vec![]; - - // Construct PhysicalSortExpr objects from Expr objects: - let mut sort_exprs = vec![]; - for expr in acc_args.sort_exprs { - if let Expr::Sort(sort) = expr { - if let Expr::Column(col) = sort.expr.as_ref() { - let name = &col.name; - let e = expressions::column::col(name, acc_args.schema)?; - sort_exprs.push(PhysicalSortExpr { - expr: e, - options: SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - }); - } - } - } - if !sort_exprs.is_empty() { - all_sort_orders.extend(sort_exprs); - } - - let ordering_req = all_sort_orders; + let ordering_req = + convert_logical_sort_exprs_to_physical(acc_args.sort_exprs, acc_args.schema)?; let ordering_dtypes = ordering_req .iter() @@ -428,30 +407,8 @@ impl AggregateUDFImpl for LastValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let mut all_sort_orders = vec![]; - - // Construct PhysicalSortExpr objects from Expr objects: - let mut sort_exprs = vec![]; - for expr in acc_args.sort_exprs { - if let Expr::Sort(sort) = expr { - if let Expr::Column(col) = sort.expr.as_ref() { - let name = &col.name; - let e = expressions::column::col(name, acc_args.schema)?; - sort_exprs.push(PhysicalSortExpr { - expr: e, - options: SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - }); - } - } - } - if !sort_exprs.is_empty() { - all_sort_orders.extend(sort_exprs); - } - - let ordering_req = all_sort_orders; + let ordering_req = + convert_logical_sort_exprs_to_physical(acc_args.sort_exprs, acc_args.schema)?; let ordering_dtypes = ordering_req .iter() @@ -472,14 +429,14 @@ impl AggregateUDFImpl for LastValue { fn state_fields(&self, args: StateFieldsArgs) -> Result> { let StateFieldsArgs { name, - input_type: _, - return_type, + input_type, + return_type: _, ordering_fields, is_distinct: _, } = args; let mut fields = vec![Field::new( format_state_name(name, "last_value"), - return_type.clone(), + input_type.clone(), true, )]; fields.extend(ordering_fields.to_vec()); @@ -715,31 +672,6 @@ fn convert_to_sort_cols( .collect::>() } -// fn replace_order_by_clause(order_by: &mut String) { -// let suffixes = [ -// (" DESC NULLS FIRST]", " ASC NULLS LAST]"), -// (" ASC NULLS FIRST]", " DESC NULLS LAST]"), -// (" DESC NULLS LAST]", " ASC NULLS FIRST]"), -// (" ASC NULLS LAST]", " DESC NULLS FIRST]"), -// ]; -// -// if let Some(start) = order_by.find("ORDER BY [") { -// if let Some(end) = order_by[start..].find(']') { -// let order_by_start = start + 9; -// let order_by_end = start + end; -// -// let column_order = &order_by[order_by_start..=order_by_end]; -// for &(suffix, replacement) in &suffixes { -// if column_order.ends_with(suffix) { -// let new_order = column_order.replace(suffix, replacement); -// order_by.replace_range(order_by_start..=order_by_end, &new_order); -// break; -// } -// } -// } -// } -// } - #[cfg(test)] mod tests { use arrow::array::Int64Array; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 611535d2ee36..51ca419bc4da 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -49,6 +49,7 @@ pub fn create_aggregate_expr( ignore_nulls: bool, is_distinct: bool, ) -> Result> { + debug_assert_eq!(sort_exprs.len(), ordering_req.len()); let input_exprs_types = input_phy_exprs .iter() .map(|arg| arg.data_type(schema)) diff --git a/datafusion/physical-expr/src/expressions/cast.rs b/datafusion/physical-expr-common/src/expressions/cast.rs similarity index 99% rename from datafusion/physical-expr/src/expressions/cast.rs rename to datafusion/physical-expr-common/src/expressions/cast.rs index 79a44ac30cfc..dca4334caaf7 100644 --- a/datafusion/physical-expr/src/expressions/cast.rs +++ b/datafusion/physical-expr-common/src/expressions/cast.rs @@ -20,8 +20,7 @@ use std::fmt; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::physical_expr::down_cast_any_ref; -use crate::PhysicalExpr; +use crate::physical_expr::{down_cast_any_ref, PhysicalExpr}; use arrow::compute::{can_cast_types, CastOptions}; use arrow::datatypes::{DataType, DataType::*, Schema}; @@ -229,7 +228,8 @@ pub fn cast( #[cfg(test)] mod tests { use super::*; - use crate::expressions::col; + + use crate::expressions::column::col; use arrow::{ array::{ diff --git a/datafusion/physical-expr-common/src/expressions/mod.rs b/datafusion/physical-expr-common/src/expressions/mod.rs index d102422081dc..4b5965e164b5 100644 --- a/datafusion/physical-expr-common/src/expressions/mod.rs +++ b/datafusion/physical-expr-common/src/expressions/mod.rs @@ -15,4 +15,7 @@ // specific language governing permissions and limitations // under the License. +mod cast; pub mod column; + +pub use cast::{cast, cast_with_options, CastExpr}; diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index 1e1187212d96..aefda7cd1d3a 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -24,10 +24,11 @@ use std::sync::Arc; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; -use datafusion_common::Result; -use datafusion_expr::ColumnarValue; +use datafusion_common::{exec_err, Result}; +use datafusion_expr::{ColumnarValue, Expr}; use crate::physical_expr::PhysicalExpr; +use crate::utils::convert_logical_expr_to_physical_expr; /// Represents Sort operation for a column in a RecordBatch #[derive(Clone, Debug)] @@ -267,3 +268,25 @@ pub type LexRequirement = Vec; ///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which /// represents a reference to a lexicographical ordering requirement. pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; + +pub fn convert_logical_sort_exprs_to_physical( + exprs: &[datafusion_expr::Expr], + schema: &Schema, +) -> Result> { + // Construct PhysicalSortExpr objects from Expr objects: + let mut sort_exprs = vec![]; + for expr in exprs { + if let Expr::Sort(sort) = expr { + sort_exprs.push(PhysicalSortExpr { + expr: convert_logical_expr_to_physical_expr(sort.expr.as_ref(), schema)?, + options: SortOptions { + descending: !sort.asc, + nulls_first: sort.nulls_first, + }, + }); + } else { + return exec_err!("Expects to receive sort expression"); + } + } + Ok(sort_exprs) +} diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index c0ee727da5d3..a371de57c96d 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -20,12 +20,15 @@ use std::sync::Arc; use datafusion_expr::Expr; use crate::{ - physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, tree_node::ExprContext, + expressions, physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, + tree_node::ExprContext, }; +use crate::expressions::CastExpr; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; -use datafusion_common::Result; +use arrow::datatypes::Schema; +use datafusion_common::{exec_err, Result}; use datafusion_expr::sort_properties::ExprProperties; /// Represents a [`PhysicalExpr`] node with associated properties (order and @@ -117,6 +120,28 @@ pub fn reverse_sort_exprs(sort_exprs: &[Expr]) -> Vec { .collect::>() } +pub fn convert_logical_expr_to_physical_expr( + expr: &datafusion_expr::Expr, + schema: &Schema, +) -> Result> { + Ok(match expr { + Expr::Column(col) => expressions::column::col(&col.name, schema)?, + Expr::Cast(cast_expr) => Arc::new(CastExpr::new( + convert_logical_expr_to_physical_expr(cast_expr.expr.as_ref(), schema)?, + cast_expr.data_type.clone(), + None, + )), + Expr::Alias(alias_expr) => { + convert_logical_expr_to_physical_expr(alias_expr.expr.as_ref(), schema)? + } + _ => { + return exec_err!( + "Unsupported expression: {expr} for conversion to Arc" + ); + } + }) +} + #[cfg(test)] mod tests { use std::sync::Arc; diff --git a/datafusion/physical-expr/src/equivalence/properties.rs b/datafusion/physical-expr/src/equivalence/properties.rs index 016c4c4ae107..7bf389ecfdf3 100644 --- a/datafusion/physical-expr/src/equivalence/properties.rs +++ b/datafusion/physical-expr/src/equivalence/properties.rs @@ -22,7 +22,7 @@ use super::ordering::collapse_lex_ordering; use crate::equivalence::{ collapse_lex_req, EquivalenceGroup, OrderingEquivalenceClass, ProjectionMapping, }; -use crate::expressions::{CastExpr, Literal}; +use crate::expressions::Literal; use crate::{ physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, @@ -35,6 +35,7 @@ use datafusion_common::{JoinSide, JoinType, Result}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::sort_properties::{ExprProperties, SortProperties}; use datafusion_physical_expr_common::expressions::column::Column; +use datafusion_physical_expr_common::expressions::CastExpr; use datafusion_physical_expr_common::utils::ExprPropertiesNode; use indexmap::{IndexMap, IndexSet}; diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 7c40f8717677..7296b1b03964 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -20,7 +20,6 @@ #[macro_use] mod binary; mod case; -mod cast; mod column; mod datum; mod in_list; @@ -78,10 +77,10 @@ pub use datafusion_functions_aggregate::first_last::{FirstValue, LastValue}; pub use binary::{binary, BinaryExpr}; pub use case::{case, CaseExpr}; -pub use cast::{cast, cast_with_options, CastExpr}; pub use column::UnKnownColumn; pub use datafusion_expr::utils::format_state_name; pub use datafusion_physical_expr_common::expressions::column::{col, Column}; +pub use datafusion_physical_expr_common::expressions::{cast, CastExpr}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index e188b2d56bae..389c8527f59d 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use crate::{ - expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, + expressions::{BinaryExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; @@ -28,6 +28,7 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{internal_datafusion_err, internal_err, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::Operator; +use datafusion_physical_expr_common::expressions::CastExpr; const MDN_DAY_MASK: i128 = 0xFFFF_FFFF_0000_0000_0000_0000; const MDN_NS_MASK: i128 = 0xFFFF_FFFF_FFFF_FFFF; diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 9e8561eb68c5..4fd2c1477795 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -27,6 +27,7 @@ use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction}; use datafusion_expr::var_provider::is_system_variables; use datafusion_expr::var_provider::VarType; use datafusion_expr::{binary_expr, Between, BinaryExpr, Expr, Like, Operator, TryCast}; +use datafusion_physical_expr_common::expressions::cast; use crate::scalar_function; use crate::{ @@ -257,7 +258,7 @@ pub fn create_physical_expr( }; Ok(expressions::case(expr, when_then_expr, else_expr)?) } - Expr::Cast(Cast { expr, data_type }) => expressions::cast( + Expr::Cast(Cast { expr, data_type }) => cast( create_physical_expr(expr, input_dfschema, execution_props)?, input_schema, data_type.clone(), diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 8d8a3e71031e..3f39c9c09bb9 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -85,6 +85,7 @@ pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, }; +pub use datafusion_physical_expr_common::expressions::{cast, CastExpr}; // Backwards compatibility pub use crate::stream::EmptyRecordBatchStream; diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index b7bc60a0486c..64cb320c4318 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -37,12 +37,12 @@ use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ - in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, - Literal, NegativeExpr, NotExpr, TryCastExpr, + in_list, BinaryExpr, CaseExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, Literal, + NegativeExpr, NotExpr, TryCastExpr, }; use datafusion::physical_plan::windows::{create_window_expr, schema_add_window_field}; use datafusion::physical_plan::{ - ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, + CastExpr, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::config::{ ColumnOptions, CsvOptions, FormatOptions, JsonOptions, ParquetOptions, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index d62eb16c878f..98d388d54bfd 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -24,17 +24,17 @@ use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ ApproxDistinct, ApproxMedian, ApproxPercentileCont, ApproxPercentileContWithWeight, - ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, - CastExpr, Column, Correlation, Count, CumeDist, DistinctArrayAgg, DistinctBitXor, - DistinctCount, DistinctSum, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, - Max, Median, Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, - OrderSensitiveArrayAgg, Rank, RankType, Regr, RegrType, RowNumber, Stddev, StddevPop, - StringAgg, Sum, TryCastExpr, Variance, VariancePop, WindowShift, + ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, Column, + Correlation, Count, CumeDist, DistinctArrayAgg, DistinctBitXor, DistinctCount, + DistinctSum, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, Max, Median, + Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, OrderSensitiveArrayAgg, + Rank, RankType, Regr, RegrType, RowNumber, Stddev, StddevPop, StringAgg, Sum, + TryCastExpr, Variance, VariancePop, WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; use datafusion::physical_plan::{ - AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, + AggregateExpr, CastExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, }; use datafusion::{ datasource::{ diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 79abecf556da..2127efaafa69 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -44,10 +44,11 @@ use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion::physical_plan::analyze::AnalyzeExec; +use datafusion::physical_plan::cast; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ - binary, cast, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, - NotExpr, NthValue, PhysicalSortExpr, StringAgg, Sum, + binary, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, NotExpr, + NthValue, PhysicalSortExpr, StringAgg, Sum, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::DataSinkExec; From 73931581fd6e4de55151da60cfb7a008d32391e1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 15:19:13 +0300 Subject: [PATCH 69/77] Minor changes --- datafusion/core/src/physical_optimizer/pruning.rs | 7 ++----- datafusion/physical-expr/src/intervals/utils.rs | 3 +-- datafusion/physical-expr/src/planner.rs | 3 +-- datafusion/physical-plan/src/lib.rs | 1 - datafusion/proto/src/physical_plan/from_proto.rs | 6 +++--- datafusion/proto/src/physical_plan/to_proto.rs | 14 +++++++------- .../proto/tests/cases/roundtrip_physical_plan.rs | 5 ++--- 7 files changed, 16 insertions(+), 23 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/pruning.rs b/datafusion/core/src/physical_optimizer/pruning.rs index 605d38896427..605ef9f9023f 100644 --- a/datafusion/core/src/physical_optimizer/pruning.rs +++ b/datafusion/core/src/physical_optimizer/pruning.rs @@ -43,7 +43,6 @@ use datafusion_common::{ }; use datafusion_physical_expr::utils::{collect_columns, Guarantee, LiteralGuarantee}; use datafusion_physical_expr::{expressions as phys_expr, PhysicalExprRef}; -use datafusion_physical_expr_common::expressions as phys_expr_common; use log::trace; @@ -1077,16 +1076,14 @@ fn rewrite_expr_to_prunable( { // `col op lit()` Ok((column_expr.clone(), op, scalar_expr.clone())) - } else if let Some(cast) = - column_expr_any.downcast_ref::() - { + } else if let Some(cast) = column_expr_any.downcast_ref::() { // `cast(col) op lit()` let arrow_schema: SchemaRef = schema.clone().into(); let from_type = cast.expr().data_type(&arrow_schema)?; verify_support_type_for_prune(&from_type, cast.cast_type())?; let (left, op, right) = rewrite_expr_to_prunable(cast.expr(), op, scalar_expr, schema)?; - let left = Arc::new(phys_expr_common::CastExpr::new( + let left = Arc::new(phys_expr::CastExpr::new( left, cast.cast_type().clone(), None, diff --git a/datafusion/physical-expr/src/intervals/utils.rs b/datafusion/physical-expr/src/intervals/utils.rs index 389c8527f59d..e188b2d56bae 100644 --- a/datafusion/physical-expr/src/intervals/utils.rs +++ b/datafusion/physical-expr/src/intervals/utils.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use crate::{ - expressions::{BinaryExpr, Column, Literal, NegativeExpr}, + expressions::{BinaryExpr, CastExpr, Column, Literal, NegativeExpr}, PhysicalExpr, }; @@ -28,7 +28,6 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::{internal_datafusion_err, internal_err, Result, ScalarValue}; use datafusion_expr::interval_arithmetic::Interval; use datafusion_expr::Operator; -use datafusion_physical_expr_common::expressions::CastExpr; const MDN_DAY_MASK: i128 = 0xFFFF_FFFF_0000_0000_0000_0000; const MDN_NS_MASK: i128 = 0xFFFF_FFFF_FFFF_FFFF; diff --git a/datafusion/physical-expr/src/planner.rs b/datafusion/physical-expr/src/planner.rs index 4fd2c1477795..9e8561eb68c5 100644 --- a/datafusion/physical-expr/src/planner.rs +++ b/datafusion/physical-expr/src/planner.rs @@ -27,7 +27,6 @@ use datafusion_expr::expr::{Alias, Cast, InList, ScalarFunction}; use datafusion_expr::var_provider::is_system_variables; use datafusion_expr::var_provider::VarType; use datafusion_expr::{binary_expr, Between, BinaryExpr, Expr, Like, Operator, TryCast}; -use datafusion_physical_expr_common::expressions::cast; use crate::scalar_function; use crate::{ @@ -258,7 +257,7 @@ pub fn create_physical_expr( }; Ok(expressions::case(expr, when_then_expr, else_expr)?) } - Expr::Cast(Cast { expr, data_type }) => cast( + Expr::Cast(Cast { expr, data_type }) => expressions::cast( create_physical_expr(expr, input_dfschema, execution_props)?, input_schema, data_type.clone(), diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 3f39c9c09bb9..8d8a3e71031e 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -85,7 +85,6 @@ pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, }; -pub use datafusion_physical_expr_common::expressions::{cast, CastExpr}; // Backwards compatibility pub use crate::stream::EmptyRecordBatchStream; diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 64cb320c4318..b7bc60a0486c 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -37,12 +37,12 @@ use datafusion::execution::FunctionRegistry; use datafusion::logical_expr::WindowFunctionDefinition; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ - in_list, BinaryExpr, CaseExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, Literal, - NegativeExpr, NotExpr, TryCastExpr, + in_list, BinaryExpr, CaseExpr, CastExpr, Column, IsNotNullExpr, IsNullExpr, LikeExpr, + Literal, NegativeExpr, NotExpr, TryCastExpr, }; use datafusion::physical_plan::windows::{create_window_expr, schema_add_window_field}; use datafusion::physical_plan::{ - CastExpr, ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, + ColumnStatistics, Partitioning, PhysicalExpr, Statistics, WindowExpr, }; use datafusion_common::config::{ ColumnOptions, CsvOptions, FormatOptions, JsonOptions, ParquetOptions, diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 98d388d54bfd..d62eb16c878f 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -24,17 +24,17 @@ use datafusion::physical_expr::window::{NthValueKind, SlidingAggregateWindowExpr use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; use datafusion::physical_plan::expressions::{ ApproxDistinct, ApproxMedian, ApproxPercentileCont, ApproxPercentileContWithWeight, - ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, Column, - Correlation, Count, CumeDist, DistinctArrayAgg, DistinctBitXor, DistinctCount, - DistinctSum, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, Max, Median, - Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, OrderSensitiveArrayAgg, - Rank, RankType, Regr, RegrType, RowNumber, Stddev, StddevPop, StringAgg, Sum, - TryCastExpr, Variance, VariancePop, WindowShift, + ArrayAgg, Avg, BinaryExpr, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, CaseExpr, + CastExpr, Column, Correlation, Count, CumeDist, DistinctArrayAgg, DistinctBitXor, + DistinctCount, DistinctSum, Grouping, InListExpr, IsNotNullExpr, IsNullExpr, Literal, + Max, Median, Min, NegativeExpr, NotExpr, NthValue, NthValueAgg, Ntile, + OrderSensitiveArrayAgg, Rank, RankType, Regr, RegrType, RowNumber, Stddev, StddevPop, + StringAgg, Sum, TryCastExpr, Variance, VariancePop, WindowShift, }; use datafusion::physical_plan::udaf::AggregateFunctionExpr; use datafusion::physical_plan::windows::{BuiltInWindowExpr, PlainAggregateWindowExpr}; use datafusion::physical_plan::{ - AggregateExpr, CastExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, + AggregateExpr, ColumnStatistics, PhysicalExpr, Statistics, WindowExpr, }; use datafusion::{ datasource::{ diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 2127efaafa69..79abecf556da 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -44,11 +44,10 @@ use datafusion::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion::physical_plan::analyze::AnalyzeExec; -use datafusion::physical_plan::cast; use datafusion::physical_plan::empty::EmptyExec; use datafusion::physical_plan::expressions::{ - binary, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, NotExpr, - NthValue, PhysicalSortExpr, StringAgg, Sum, + binary, cast, col, in_list, like, lit, Avg, BinaryExpr, Column, DistinctCount, + NotExpr, NthValue, PhysicalSortExpr, StringAgg, Sum, }; use datafusion::physical_plan::filter::FilterExec; use datafusion::physical_plan::insert::DataSinkExec; From 92fec67c1d07325fbaa3fbc772291e6782fdd56d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 15:20:03 +0300 Subject: [PATCH 70/77] Fix test --- datafusion/sqllogictest/test_files/distinct_on.slt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 97b988914d82..cdef2990fa3c 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -116,8 +116,8 @@ SELECT DISTINCT ON () c1, c2 FROM aggregate_test_100 ORDER BY c1, c2; query II SELECT DISTINCT ON (c2 % 2 = 0) c2, c3 - 100 FROM aggregate_test_100 ORDER BY c2 % 2 = 0, c3 DESC; ---- -5 -140 -2 -99 +1 25 +4 23 # Multiple complex expressions query TIB @@ -127,7 +127,7 @@ WHERE c1 IN ('a', 'b') ORDER BY chr(ascii(c1) + 3), c2 % 2, c3 DESC; ---- D 0 false -D 1 false +D 1 true E 0 false E 1 false From 809dc7f2aae588c40e69db98242b365da57fda7d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 15:23:03 +0300 Subject: [PATCH 71/77] Minor changes --- datafusion/functions/Cargo.toml | 2 +- datafusion/proto/src/generated/prost.rs | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/datafusion/functions/Cargo.toml b/datafusion/functions/Cargo.toml index e4e99beee97f..efc12e71a9ac 100644 --- a/datafusion/functions/Cargo.toml +++ b/datafusion/functions/Cargo.toml @@ -80,7 +80,7 @@ itertools = { workspace = true } log = { workspace = true } md-5 = { version = "^0.10.0", optional = true } rand = { workspace = true } -regex = { worksapce = true, optional = true } +regex = { workspace = true, optional = true } sha2 = { version = "^0.10.1", optional = true } unicode-segmentation = { version = "^1.7.1", optional = true } uuid = { version = "1.7", features = ["v4"], optional = true } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index dff7088434b3..4f17e6c58728 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2854,10 +2854,6 @@ pub enum AggregateFunction { BitXor = 21, BoolAnd = 22, BoolOr = 23, - /// // 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; RegrSlope = 26, RegrIntercept = 27, RegrCount = 28, From e091afcc501eca3e996426c3dffe9b9d28e7ba91 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 23 May 2024 15:56:00 +0300 Subject: [PATCH 72/77] Simplifications --- datafusion/expr/src/udaf.rs | 13 +- .../physical-expr-common/src/aggregate/mod.rs | 4 +- .../physical-expr-common/src/sort_expr.rs | 2 + datafusion/physical-expr-common/src/utils.rs | 7 +- .../physical-plan/src/aggregates/mod.rs | 136 +++++++++--------- 5 files changed, 84 insertions(+), 78 deletions(-) diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 033d368a1f0f..f1f5c0baa812 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -194,6 +194,9 @@ impl AggregateUDF { self.inner.create_groups_accumulator() } + /// Sets the flag specifying whether the requirement of the UDf is satisfied or not. + /// `None` indicates that, UDF doesn't have support for requirement satisfied mode (In this case, its requirement + /// should be definitely satisfied). pub fn with_requirement_satisfied( self, requirement_satisfied: bool, @@ -204,10 +207,13 @@ impl AggregateUDF { Ok(updated_udf.map(|udf| Self { inner: udf })) } + /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] for possible options. pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { self.inner.order_sensitivity() } + /// Reserves the `AggregateUDF` (e.g. returns the `AggregateUDF` that will generate same result with this + /// `AggregateUDF` when iterated in reverse order. `None` indicates, there is no such `AggregateUDF`.) pub fn reverse_udf(&self) -> Option { match self.inner.reverse_expr() { ReversedUDAF::NotSupported => None, @@ -383,6 +389,9 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { &[] } + /// Sets the flag specifying whether the requirement of the UDf is satisfied or not. + /// `None` indicates that, UDF doesn't have support for requirement satisfied mode (In this case, its requirement + /// should be definitely satisfied). fn with_requirement_satisfied( self: Arc, _requirement_satisfied: bool, @@ -391,8 +400,10 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { Ok(None) } + /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] for possible options. fn order_sensitivity(&self) -> AggregateOrderSensitivity { - // By default, requirement is hard if not specified. + // By default, requirement is hard. This means, requirement should be definitely satisfied (if any). + // For aggregator to generate correct result. AggregateOrderSensitivity::HardRequirement } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 51ca419bc4da..dc3ef61a851d 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -338,7 +338,7 @@ impl AggregateExpr for AggregateFunctionExpr { fn order_sensitivity(&self) -> AggregateOrderSensitivity { if !self.ordering_req.is_empty() { - // If there is requirement, use the sensitive of the implementation + // If there is requirement, use the sensitivity of the implementation self.fun.order_sensitivity() } else { // If no requirement, aggregator is order insensitive @@ -376,8 +376,6 @@ impl AggregateExpr for AggregateFunctionExpr { let reverse_ordering_req = reverse_order_bys(&self.ordering_req); let reverse_sort_exprs = reverse_sort_exprs(&self.sort_exprs); let mut name = self.name().to_string(); - // let name = calc_fn_name_with_args(self.fun.name(), &self.args); - // let name = self.fun.name(); replace_order_by_clause(&mut name); replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); let reverse_aggr = create_aggregate_expr( diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index aefda7cd1d3a..bce2078ac17d 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -269,6 +269,8 @@ pub type LexRequirement = Vec; /// represents a reference to a lexicographical ordering requirement. pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; +/// Converts each `datafusion_expr::Expr` into corresponding `PhysicalSortExpr`. +/// Assumes `datafusion_expr::Expr` is `datafusion_expr::Expr::Sort` otherwise returns Error. pub fn convert_logical_sort_exprs_to_physical( exprs: &[datafusion_expr::Expr], schema: &Schema, diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index a371de57c96d..bb9ca71d22fa 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -17,8 +17,6 @@ use std::sync::Arc; -use datafusion_expr::Expr; - use crate::{ expressions, physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, tree_node::ExprContext, @@ -30,6 +28,7 @@ use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::Schema; use datafusion_common::{exec_err, Result}; use datafusion_expr::sort_properties::ExprProperties; +use datafusion_expr::Expr; /// Represents a [`PhysicalExpr`] node with associated properties (order and /// range) in a context where properties are tracked. @@ -113,13 +112,15 @@ pub fn reverse_sort_exprs(sort_exprs: &[Expr]) -> Vec { if let Expr::Sort(s) = e { Expr::Sort(s.reverse()) } else { - // TODO: Return error + // Expects to receive `Expr::Sort`. unreachable!() } }) .collect::>() } +/// Converts `datafusion_expr::Expr` into corresponding `Arc`. +/// If conversion is not supported yet, returns Error. pub fn convert_logical_expr_to_physical_expr( expr: &datafusion_expr::Expr, schema: &Schema, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 1eaad3e3314d..50a31e11a2ce 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1202,7 +1202,7 @@ mod tests { use datafusion_physical_expr::expressions::{ lit, ApproxDistinct, Count, FirstValue, LastValue, Median, OrderSensitiveArrayAgg, }; - use datafusion_physical_expr::{reverse_order_bys, PhysicalSortExpr}; + use datafusion_physical_expr::PhysicalSortExpr; use futures::{FutureExt, Stream}; @@ -1941,6 +1941,66 @@ mod tests { Ok(()) } + // FIRST_VALUE(b ORDER BY b ) + fn test_first_value_agg_expr( + schema: &Schema, + sort_options: SortOptions, + ) -> Result> { + let sort_exprs = vec![datafusion_expr::Expr::Sort(Sort { + expr: Box::new(datafusion_expr::Expr::Column( + datafusion_common::Column::new(Some("table1"), "b"), + )), + asc: !sort_options.descending, + nulls_first: sort_options.nulls_first, + })]; + let ordering_req = vec![PhysicalSortExpr { + expr: col("b", schema)?, + options: sort_options, + }]; + let args = vec![col("b", schema)?]; + let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); + datafusion_physical_expr_common::aggregate::create_aggregate_expr( + &func, + &args, + &sort_exprs, + &ordering_req, + schema, + "FIRST_VALUE(b)", + false, + false, + ) + } + + // LAST_VALUE(b ORDER BY b ) + fn test_last_value_agg_expr( + schema: &Schema, + sort_options: SortOptions, + ) -> Result> { + let sort_exprs = vec![datafusion_expr::Expr::Sort(Sort { + expr: Box::new(datafusion_expr::Expr::Column( + datafusion_common::Column::new(Some("table1"), "b"), + )), + asc: !sort_options.descending, + nulls_first: sort_options.nulls_first, + })]; + let ordering_req = vec![PhysicalSortExpr { + expr: col("b", schema)?, + options: sort_options, + }]; + let args = vec![col("b", schema)?]; + let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); + datafusion_physical_expr_common::aggregate::create_aggregate_expr( + &func, + &args, + &sort_exprs, + &ordering_req, + schema, + "LAST_VALUE(b)", + false, + false, + ) + } + // This function either constructs the physical plan below, // // "AggregateExec: mode=Final, gby=[a@0 as a], aggr=[FIRST_VALUE(b)]", @@ -1982,48 +2042,10 @@ mod tests { descending: false, nulls_first: false, }; - let sort_exprs = vec![datafusion_expr::Expr::Sort(Sort { - expr: Box::new(datafusion_expr::Expr::Column( - datafusion_common::Column::new(Some("table1"), "b"), - )), - asc: !sort_options.descending, - nulls_first: sort_options.nulls_first, - })]; - let ordering_req = vec![PhysicalSortExpr { - expr: col("b", &schema)?, - options: sort_options, - }]; - let args = vec![col("b", &schema)?]; let aggregates: Vec> = if is_first_acc { - let func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - vec![ - datafusion_physical_expr_common::aggregate::create_aggregate_expr( - &func, - &args, - &sort_exprs, - &ordering_req, - &schema, - "FIRST_VALUE(b)", - false, - false, - ) - .unwrap(), - ] + vec![test_first_value_agg_expr(&schema, sort_options)?] } else { - let func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); - vec![ - datafusion_physical_expr_common::aggregate::create_aggregate_expr( - &func, - &args, - &sort_exprs, - &ordering_req, - &schema, - "LAST_VALUE(b)", - false, - false, - ) - .unwrap(), - ] + vec![test_last_value_agg_expr(&schema, sort_options)?] }; let memory_exec = Arc::new(MemoryExec::try_new( @@ -2178,43 +2200,15 @@ mod tests { ])); let col_a = col("a", &schema)?; - let col_b = col("b", &schema)?; let option_desc = SortOptions { descending: true, nulls_first: true, }; - let sort_expr = vec![PhysicalSortExpr { - expr: col_b.clone(), - options: option_desc, - }]; - let sort_expr_reverse = reverse_order_bys(&sort_expr); let groups = PhysicalGroupBy::new_single(vec![(col_a, "a".to_string())]); - let first_func = datafusion_expr::AggregateUDF::new_from_impl(FirstValue::new()); - let last_func = datafusion_expr::AggregateUDF::new_from_impl(LastValue::new()); let aggregates: Vec> = vec![ - datafusion_physical_expr_common::aggregate::create_aggregate_expr( - &first_func, - &[col_b.clone()], - &[], - &sort_expr_reverse, - &schema, - "FIRST_VALUE(b)", - false, - false, - ) - .unwrap(), - datafusion_physical_expr_common::aggregate::create_aggregate_expr( - &last_func, - &[col_b.clone()], - &[], - &sort_expr, - &schema, - "LAST_VALUE(b)", - false, - false, - ) - .unwrap(), + test_first_value_agg_expr(&schema, option_desc)?, + test_last_value_agg_expr(&schema, option_desc)?, ]; let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); let aggregate_exec = Arc::new(AggregateExec::try_new( From 89a601bb65afee5f25de3e92abfaecc970241409 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 24 May 2024 13:34:35 +0300 Subject: [PATCH 73/77] Review --- .../physical_optimizer/update_aggr_exprs.rs | 29 +++--- datafusion/expr/src/lib.rs | 3 +- datafusion/expr/src/udaf.rs | 37 ++++---- datafusion/expr/src/utils.rs | 22 ++--- .../functions-aggregate/src/first_last.rs | 4 +- .../src/replace_distinct_aggregate.rs | 5 +- .../physical-expr-common/src/aggregate/mod.rs | 88 +++++++++++-------- .../physical-expr-common/src/sort_expr.rs | 31 ++++--- datafusion/physical-expr-common/src/utils.rs | 50 ++++------- .../src/aggregate/array_agg_ordered.rs | 7 +- .../physical-expr/src/aggregate/build_in.rs | 18 ++-- .../physical-expr/src/expressions/mod.rs | 22 +++-- .../physical-plan/src/aggregates/mod.rs | 19 ++-- 13 files changed, 157 insertions(+), 178 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index bdebd547e091..11411e48b18d 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -134,11 +134,6 @@ fn try_convert_aggregate_if_better( prefix_requirement: &[PhysicalSortRequirement], eq_properties: &EquivalenceProperties, ) -> Result>> { - let err_fn = || { - plan_datafusion_err!( - "Expects an aggregate expression that can benefit from input ordering" - ) - }; aggr_exprs .into_iter() .map(|aggr_expr| { @@ -153,32 +148,32 @@ fn try_convert_aggregate_if_better( // there is an actual ordering enabling this, try to update the // aggregate expression to benefit from the existing ordering. // Otherwise, leave it as is. - if aggr_expr.order_sensitivity().is_order_beneficial() - && !aggr_sort_reqs.is_empty() + if aggr_expr.order_sensitivity().is_beneficial() && !aggr_sort_reqs.is_empty() { let reqs = concat_slices(prefix_requirement, &aggr_sort_reqs); if eq_properties.ordering_satisfy_requirement(&reqs) { // Existing ordering satisfies the aggregator requirements: - return aggr_expr - .with_requirement_satisfied(true)? - .ok_or_else(err_fn); - } - let reqs = concat_slices(prefix_requirement, &reverse_aggr_req); - if eq_properties.ordering_satisfy_requirement(&reqs) { + aggr_expr.with_requirement_satisfied(true)? + } else if eq_properties.ordering_satisfy_requirement(&concat_slices( + prefix_requirement, + &reverse_aggr_req, + )) { // Converting to reverse enables more efficient execution // given the existing ordering (if possible): aggr_expr .reverse_expr() .unwrap_or(aggr_expr) .with_requirement_satisfied(true)? - .ok_or_else(err_fn) } else { // There is no beneficial ordering present -- aggregation // will still work albeit in a less efficient mode. - aggr_expr - .with_requirement_satisfied(false)? - .ok_or_else(err_fn) + aggr_expr.with_requirement_satisfied(false)? } + .ok_or_else(|| { + plan_datafusion_err!( + "Expects an aggregate expression that can benefit from input ordering" + ) + }) } else { Ok(aggr_expr) } diff --git a/datafusion/expr/src/lib.rs b/datafusion/expr/src/lib.rs index f343f28fd3f5..5441329f3490 100644 --- a/datafusion/expr/src/lib.rs +++ b/datafusion/expr/src/lib.rs @@ -81,8 +81,7 @@ pub use signature::{ ArrayFunctionSignature, Signature, TypeSignature, Volatility, TIMEZONE_WILDCARD, }; pub use table_source::{TableProviderFilterPushDown, TableSource, TableType}; -pub use udaf::ReversedUDAF; -pub use udaf::{AggregateUDF, AggregateUDFImpl}; +pub use udaf::{AggregateUDF, AggregateUDFImpl, ReversedUDAF}; pub use udf::{ScalarUDF, ScalarUDFImpl}; pub use udwf::{WindowUDF, WindowUDFImpl}; pub use window_frame::{WindowFrame, WindowFrameBound, WindowFrameUnits}; diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index f1f5c0baa812..c999ca979312 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -194,26 +194,28 @@ impl AggregateUDF { self.inner.create_groups_accumulator() } - /// Sets the flag specifying whether the requirement of the UDf is satisfied or not. - /// `None` indicates that, UDF doesn't have support for requirement satisfied mode (In this case, its requirement - /// should be definitely satisfied). + /// Sets the flag specifying whether the requirement of the UDF is satisfied. + /// A return value of `None` indicates that the UDF doesn't have support for + /// requirement satisfied mode (In this case, its requirement should be + /// definitely satisfied). pub fn with_requirement_satisfied( self, requirement_satisfied: bool, ) -> Result> { - let updated_udf = self - .inner - .with_requirement_satisfied(requirement_satisfied)?; - Ok(updated_udf.map(|udf| Self { inner: udf })) + self.inner + .with_requirement_satisfied(requirement_satisfied) + .map(|updated_udf| updated_udf.map(|udf| Self { inner: udf })) } - /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] for possible options. + /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] + /// for possible options. pub fn order_sensitivity(&self) -> AggregateOrderSensitivity { self.inner.order_sensitivity() } - /// Reserves the `AggregateUDF` (e.g. returns the `AggregateUDF` that will generate same result with this - /// `AggregateUDF` when iterated in reverse order. `None` indicates, there is no such `AggregateUDF`.) + /// Reserves the `AggregateUDF` (e.g. returns the `AggregateUDF` that will + /// generate same result with this `AggregateUDF` when iterated in reverse + /// order, and `None` if there is no such `AggregateUDF`). pub fn reverse_udf(&self) -> Option { match self.inner.reverse_expr() { ReversedUDAF::NotSupported => None, @@ -389,9 +391,10 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { &[] } - /// Sets the flag specifying whether the requirement of the UDf is satisfied or not. - /// `None` indicates that, UDF doesn't have support for requirement satisfied mode (In this case, its requirement - /// should be definitely satisfied). + /// Sets the flag specifying whether the requirement of the UDF is satisfied. + /// A return value of `None` indicates that the UDF doesn't have support for + /// requirement satisfied mode (In this case, its requirement should be + /// definitely satisfied). fn with_requirement_satisfied( self: Arc, _requirement_satisfied: bool, @@ -400,10 +403,12 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { Ok(None) } - /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] for possible options. + /// Gets the order sensitivity of the UDF. See [`AggregateOrderSensitivity`] + /// for possible options. fn order_sensitivity(&self) -> AggregateOrderSensitivity { - // By default, requirement is hard. This means, requirement should be definitely satisfied (if any). - // For aggregator to generate correct result. + // We have hard ordering requirements by default, meaning that order + // sensitive UDFs need their input orderings to satisfy their ordering + // requirements to generate correct results. AggregateOrderSensitivity::HardRequirement } diff --git a/datafusion/expr/src/utils.rs b/datafusion/expr/src/utils.rs index 04fbcdb83abf..e5b7bddab837 100644 --- a/datafusion/expr/src/utils.rs +++ b/datafusion/expr/src/utils.rs @@ -1220,28 +1220,30 @@ pub fn format_state_name(name: &str, state_name: &str) -> String { /// Represents the sensitivity of an aggregate expression to ordering. #[derive(Debug, PartialEq, Eq, Clone, Copy)] pub enum AggregateOrderSensitivity { - /// Indicates that the aggregate expression is insensitive to ordering. Ordering at the input - /// is not important for the result of the aggregator + /// Indicates that the aggregate expression is insensitive to ordering. + /// Ordering at the input is not important for the result of the aggregator. Insensitive, - /// Indicates that the aggregate expression has a hard requirement on ordering. Aggregator cannot produce - /// correct result unless its ordering requirement is satisfied. + /// Indicates that the aggregate expression has a hard requirement on ordering. + /// The aggregator can not produce a correct result unless its ordering + /// requirement is satisfied. HardRequirement, - /// Indicates that ordering is beneficial for the aggregate expression. Aggregator can produce its result efficiently - /// when its required ordering is satisfied. However, it can still produce correct result (less efficiently) - /// when its required ordering is not satisfied. + /// Indicates that ordering is beneficial for the aggregate expression in terms + /// of evaluation efficiency. The aggregator can produce its result efficiently + /// when its required ordering is satisfied; however, it can still produce the + /// correct result (albeit less efficiently) when its required ordering is not met. Beneficial, } impl AggregateOrderSensitivity { - pub fn is_order_insensitive(&self) -> bool { + pub fn is_insensitive(&self) -> bool { self.eq(&AggregateOrderSensitivity::Insensitive) } - pub fn is_order_beneficial(&self) -> bool { + pub fn is_beneficial(&self) -> bool { self.eq(&AggregateOrderSensitivity::Beneficial) } - pub fn is_order_hard_required(&self) -> bool { + pub fn hard_requires(&self) -> bool { self.eq(&AggregateOrderSensitivity::HardRequirement) } } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 204713282228..20e49df0ad77 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -117,8 +117,8 @@ impl AggregateUDFImpl for FirstValue { .map(|e| e.expr.data_type(acc_args.schema)) .collect::>>()?; - // When requirement is empty, or it is signalled by outside caller: - // accumulator assumes ordering requirement is satisfied. + // When requirement is empty, or it is signalled by outside caller that + // the ordering requirement is/will be satisfied. let requirement_satisfied = ordering_req.is_empty() || self.requirement_satisfied; FirstValueAccumulator::try_new( diff --git a/datafusion/optimizer/src/replace_distinct_aggregate.rs b/datafusion/optimizer/src/replace_distinct_aggregate.rs index 69a9d087259e..c232935f9e23 100644 --- a/datafusion/optimizer/src/replace_distinct_aggregate.rs +++ b/datafusion/optimizer/src/replace_distinct_aggregate.rs @@ -97,10 +97,7 @@ impl OptimizerRule for ReplaceDistinctWithAggregate { // Construct the aggregation expression to be used to fetch the selected expressions. let aggr_expr = select_expr .into_iter() - .map(|e| { - first_value(vec![e.clone()], false, None, sort_expr.clone(), None) - }) - .collect::>(); + .map(|e| first_value(vec![e], false, None, sort_expr.clone(), None)); let aggr_expr = normalize_cols(aggr_expr, input.as_ref())?; let group_expr = normalize_cols(on_expr, input.as_ref())?; diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index dc3ef61a851d..460c6d5ec1c2 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -19,22 +19,22 @@ pub mod groups_accumulator; pub mod stats; pub mod utils; +use std::fmt::Debug; +use std::{any::Any, sync::Arc}; + +use self::utils::{down_cast_any_ref, ordering_fields}; +use crate::physical_expr::PhysicalExpr; +use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; +use crate::utils::reverse_order_bys; + use arrow::datatypes::{DataType, Field, Schema}; -use datafusion_common::{exec_datafusion_err, not_impl_err, Result}; +use datafusion_common::{exec_err, not_impl_err, Result}; use datafusion_expr::function::StateFieldsArgs; use datafusion_expr::type_coercion::aggregates::check_arg_count; use datafusion_expr::utils::AggregateOrderSensitivity; use datafusion_expr::{ function::AccumulatorArgs, Accumulator, AggregateUDF, Expr, GroupsAccumulator, }; -use std::fmt::Debug; -use std::{any::Any, sync::Arc}; - -use crate::physical_expr::PhysicalExpr; -use crate::sort_expr::{LexOrdering, PhysicalSortExpr}; -use crate::utils::{reverse_order_bys, reverse_sort_exprs}; - -use self::utils::{down_cast_any_ref, ordering_fields}; /// Creates a physical expression of the UDAF, that includes all necessary type coercion. /// This function errors when `args`' can't be coerced to a valid argument type of the UDAF. @@ -120,31 +120,33 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { None } - /// Indicates whether aggregator can produce correct result with any arbitrary ordering or not. - /// By default we assume aggregate expression is order insensitive. + /// Indicates whether aggregator can produce the correct result with any + /// arbitrary input ordering. By default, we assume that aggregate expressions + /// are order insensitive. fn order_sensitivity(&self) -> AggregateOrderSensitivity { AggregateOrderSensitivity::Insensitive } - /// Sets the indicator whether requirement of the aggregators is satisfied at the input. - /// If this is not the case: Aggregators with order sensitivity `AggregateOrderSensitivity::Beneficial` can still produce - /// correct result with possibly more work internally. + /// Sets the indicator whether ordering requirements of the aggregator is + /// satisfied by its input. If this is not the case, aggregators with order + /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce + /// the correct result with possibly more work internally. /// /// # Returns /// /// Returns `Ok(Some(updated_expr))` if the process completes successfully. - /// If the expression which can benefit does not implement the method, it returns an error. - /// [`AggregateOrderSensitivity::Insensitive`] and [`AggregateOrderSensitivity::HardRequirement`] - /// expressions return Ok(None). + /// If the expression can benefit from existing input ordering, but does + /// not implement the method, returns an error. Order insensitive and hard + /// requirement aggregators return `Ok(None)`. fn with_requirement_satisfied( self: Arc, _requirement_satisfied: bool, ) -> Result>> { - if self.order_bys().is_some() && self.order_sensitivity().is_order_beneficial() { - return Err(exec_datafusion_err!( + if self.order_bys().is_some() && self.order_sensitivity().is_beneficial() { + return exec_err!( "Should implement with satisfied for aggregator :{:?}", self.name() - )); + ); } Ok(None) } @@ -350,31 +352,41 @@ impl AggregateExpr for AggregateFunctionExpr { self: Arc, requirement_satisfied: bool, ) -> Result>> { - if let Some(updated_fn) = self + let Some(updated_fn) = self .fun .clone() .with_requirement_satisfied(requirement_satisfied)? - { - let aggr_expr = create_aggregate_expr( - &updated_fn, - &self.args, - &self.sort_exprs, - &self.ordering_req, - &self.schema, - self.name(), - self.ignore_nulls, - self.is_distinct, - ) - .unwrap(); - return Ok(Some(aggr_expr)); - } - Ok(None) + else { + return Ok(None); + }; + create_aggregate_expr( + &updated_fn, + &self.args, + &self.sort_exprs, + &self.ordering_req, + &self.schema, + self.name(), + self.ignore_nulls, + self.is_distinct, + ) + .map(Some) } fn reverse_expr(&self) -> Option> { if let Some(reverse_udf) = self.fun.reverse_udf() { let reverse_ordering_req = reverse_order_bys(&self.ordering_req); - let reverse_sort_exprs = reverse_sort_exprs(&self.sort_exprs); + let reverse_sort_exprs = self + .sort_exprs + .iter() + .map(|e| { + if let Expr::Sort(s) = e { + Expr::Sort(s.reverse()) + } else { + // Expects to receive `Expr::Sort`. + unreachable!() + } + }) + .collect::>(); let mut name = self.name().to_string(); replace_order_by_clause(&mut name); replace_fn_name_clause(&mut name, self.fun.name(), reverse_udf.name()); @@ -428,7 +440,7 @@ fn replace_order_by_clause(order_by: &mut String) { let order_by_end = start + end; let column_order = &order_by[order_by_start..=order_by_end]; - for &(suffix, replacement) in &suffixes { + for (suffix, replacement) in suffixes { if column_order.ends_with(suffix) { let new_order = column_order.replace(suffix, replacement); order_by.replace_range(order_by_start..=order_by_end, &new_order); diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index bce2078ac17d..da55f9398087 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -21,15 +21,15 @@ use std::fmt::Display; use std::hash::{Hash, Hasher}; use std::sync::Arc; +use crate::physical_expr::PhysicalExpr; +use crate::utils::convert_logical_expr_to_physical_expr; + use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use datafusion_common::{exec_err, Result}; use datafusion_expr::{ColumnarValue, Expr}; -use crate::physical_expr::PhysicalExpr; -use crate::utils::convert_logical_expr_to_physical_expr; - /// Represents Sort operation for a column in a RecordBatch #[derive(Clone, Debug)] pub struct PhysicalSortExpr { @@ -269,26 +269,25 @@ pub type LexRequirement = Vec; /// represents a reference to a lexicographical ordering requirement. pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; -/// Converts each `datafusion_expr::Expr` into corresponding `PhysicalSortExpr`. -/// Assumes `datafusion_expr::Expr` is `datafusion_expr::Expr::Sort` otherwise returns Error. +/// Converts each [`Expr::Sort`] into a corresponding [`PhysicalSortExpr`]. +/// Returns an error if the given logical expression is not a [`Expr::Sort`]. pub fn convert_logical_sort_exprs_to_physical( - exprs: &[datafusion_expr::Expr], + exprs: &[Expr], schema: &Schema, ) -> Result> { // Construct PhysicalSortExpr objects from Expr objects: let mut sort_exprs = vec![]; for expr in exprs { - if let Expr::Sort(sort) = expr { - sort_exprs.push(PhysicalSortExpr { - expr: convert_logical_expr_to_physical_expr(sort.expr.as_ref(), schema)?, - options: SortOptions { - descending: !sort.asc, - nulls_first: sort.nulls_first, - }, - }); - } else { + let Expr::Sort(sort) = expr else { return exec_err!("Expects to receive sort expression"); - } + }; + sort_exprs.push(PhysicalSortExpr { + expr: convert_logical_expr_to_physical_expr(sort.expr.as_ref(), schema)?, + options: SortOptions { + descending: !sort.asc, + nulls_first: sort.nulls_first, + }, + }); } Ok(sort_exprs) } diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index bb9ca71d22fa..7e35534d67bb 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -17,12 +17,11 @@ use std::sync::Arc; -use crate::{ - expressions, physical_expr::PhysicalExpr, sort_expr::PhysicalSortExpr, - tree_node::ExprContext, -}; +use crate::expressions::{self, CastExpr}; +use crate::physical_expr::PhysicalExpr; +use crate::sort_expr::PhysicalSortExpr; +use crate::tree_node::ExprContext; -use crate::expressions::CastExpr; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::Schema; @@ -104,54 +103,37 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec Vec { - sort_exprs - .iter() - .map(|e| { - if let Expr::Sort(s) = e { - Expr::Sort(s.reverse()) - } else { - // Expects to receive `Expr::Sort`. - unreachable!() - } - }) - .collect::>() -} - /// Converts `datafusion_expr::Expr` into corresponding `Arc`. /// If conversion is not supported yet, returns Error. pub fn convert_logical_expr_to_physical_expr( - expr: &datafusion_expr::Expr, + expr: &Expr, schema: &Schema, ) -> Result> { - Ok(match expr { - Expr::Column(col) => expressions::column::col(&col.name, schema)?, - Expr::Cast(cast_expr) => Arc::new(CastExpr::new( + match expr { + Expr::Column(col) => expressions::column::col(&col.name, schema), + Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( convert_logical_expr_to_physical_expr(cast_expr.expr.as_ref(), schema)?, cast_expr.data_type.clone(), None, - )), + ))), Expr::Alias(alias_expr) => { - convert_logical_expr_to_physical_expr(alias_expr.expr.as_ref(), schema)? - } - _ => { - return exec_err!( - "Unsupported expression: {expr} for conversion to Arc" - ); + convert_logical_expr_to_physical_expr(alias_expr.expr.as_ref(), schema) } - }) + _ => exec_err!( + "Unsupported expression: {expr} for conversion to Arc" + ), + } } #[cfg(test)] mod tests { use std::sync::Arc; + use super::*; + use arrow::array::Int32Array; use datafusion_common::cast::{as_boolean_array, as_int32_array}; - use super::*; - #[test] fn scatter_int() -> Result<()> { let truthy = Arc::new(Int32Array::from(vec![1, 10, 11, 100])); diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 78eac79f20f8..837a9d551153 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -30,14 +30,11 @@ use crate::{ reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, }; -use arrow::array::{Array, ArrayRef}; use arrow::datatypes::{DataType, Field}; use arrow_array::cast::AsArray; -use arrow_array::{new_empty_array, StructArray}; +use arrow_array::{new_empty_array, Array, ArrayRef, StructArray}; use arrow_schema::{Fields, SortOptions}; - -use datafusion_common::utils::array_into_list_array; -use datafusion_common::utils::{compare_rows, get_row_at_idx}; +use datafusion_common::utils::{array_into_list_array, compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, Result, ScalarValue}; use datafusion_expr::utils::AggregateOrderSensitivity; use datafusion_expr::Accumulator; diff --git a/datafusion/physical-expr/src/aggregate/build_in.rs b/datafusion/physical-expr/src/aggregate/build_in.rs index 3fe40a12f7a0..3c706df0df9c 100644 --- a/datafusion/physical-expr/src/aggregate/build_in.rs +++ b/datafusion/physical-expr/src/aggregate/build_in.rs @@ -28,15 +28,14 @@ use std::sync::Arc; -use arrow::datatypes::Schema; - -use datafusion_common::{exec_err, not_impl_err, Result}; -use datafusion_expr::AggregateFunction; - use crate::aggregate::regr::RegrType; use crate::expressions::{self, Literal}; use crate::{AggregateExpr, PhysicalExpr, PhysicalSortExpr}; +use arrow::datatypes::Schema; +use datafusion_common::{exec_err, not_impl_err, Result}; +use datafusion_expr::AggregateFunction; + /// Create a physical aggregation expression. /// This function errors when `input_phy_exprs`' can't be coerced to a valid argument type of the aggregation function. pub fn create_aggregate_expr( @@ -381,17 +380,16 @@ pub fn create_aggregate_expr( mod tests { use arrow::datatypes::{DataType, Field}; - use datafusion_common::{plan_err, DataFusionError, ScalarValue}; - use datafusion_expr::type_coercion::aggregates::NUMERICS; - use datafusion_expr::{type_coercion, Signature}; - + use super::*; use crate::expressions::{ try_cast, ApproxDistinct, ApproxMedian, ApproxPercentileCont, ArrayAgg, Avg, BitAnd, BitOr, BitXor, BoolAnd, BoolOr, Count, DistinctArrayAgg, DistinctCount, Max, Min, Stddev, Sum, Variance, }; - use super::*; + use datafusion_common::{plan_err, DataFusionError, ScalarValue}; + use datafusion_expr::type_coercion::aggregates::NUMERICS; + use datafusion_expr::{type_coercion, Signature}; #[test] fn test_count_arragg_approx_expr() -> Result<()> { diff --git a/datafusion/physical-expr/src/expressions/mod.rs b/datafusion/physical-expr/src/expressions/mod.rs index 7296b1b03964..d25bd862745e 100644 --- a/datafusion/physical-expr/src/expressions/mod.rs +++ b/datafusion/physical-expr/src/expressions/mod.rs @@ -53,8 +53,7 @@ pub use crate::aggregate::count::Count; pub use crate::aggregate::count_distinct::DistinctCount; pub use crate::aggregate::grouping::Grouping; pub use crate::aggregate::median::Median; -pub use crate::aggregate::min_max::{Max, Min}; -pub use crate::aggregate::min_max::{MaxAccumulator, MinAccumulator}; +pub use crate::aggregate::min_max::{Max, MaxAccumulator, Min, MinAccumulator}; pub use crate::aggregate::nth_value::NthValueAgg; pub use crate::aggregate::regr::{Regr, RegrType}; pub use crate::aggregate::stats::StatsType; @@ -63,24 +62,17 @@ pub use crate::aggregate::string_agg::StringAgg; pub use crate::aggregate::sum::Sum; pub use crate::aggregate::sum_distinct::DistinctSum; pub use crate::aggregate::variance::{Variance, VariancePop}; -pub use crate::window::cume_dist::cume_dist; -pub use crate::window::cume_dist::CumeDist; -pub use crate::window::lead_lag::WindowShift; -pub use crate::window::lead_lag::{lag, lead}; +pub use crate::window::cume_dist::{cume_dist, CumeDist}; +pub use crate::window::lead_lag::{lag, lead, WindowShift}; pub use crate::window::nth_value::NthValue; pub use crate::window::ntile::Ntile; -pub use crate::window::rank::{dense_rank, percent_rank, rank}; -pub use crate::window::rank::{Rank, RankType}; +pub use crate::window::rank::{dense_rank, percent_rank, rank, Rank, RankType}; pub use crate::window::row_number::RowNumber; pub use crate::PhysicalSortExpr; -pub use datafusion_functions_aggregate::first_last::{FirstValue, LastValue}; pub use binary::{binary, BinaryExpr}; pub use case::{case, CaseExpr}; pub use column::UnKnownColumn; -pub use datafusion_expr::utils::format_state_name; -pub use datafusion_physical_expr_common::expressions::column::{col, Column}; -pub use datafusion_physical_expr_common::expressions::{cast, CastExpr}; pub use in_list::{in_list, InListExpr}; pub use is_not_null::{is_not_null, IsNotNullExpr}; pub use is_null::{is_null, IsNullExpr}; @@ -91,11 +83,17 @@ pub use no_op::NoOp; pub use not::{not, NotExpr}; pub use try_cast::{try_cast, TryCastExpr}; +pub use datafusion_expr::utils::format_state_name; +pub use datafusion_functions_aggregate::first_last::{FirstValue, LastValue}; +pub use datafusion_physical_expr_common::expressions::column::{col, Column}; +pub use datafusion_physical_expr_common::expressions::{cast, CastExpr}; + #[cfg(test)] pub(crate) mod tests { use std::sync::Arc; use crate::AggregateExpr; + use arrow::record_batch::RecordBatch; use datafusion_common::{Result, ScalarValue}; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 50a31e11a2ce..ab55584e7344 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -39,14 +39,11 @@ use datafusion_common::stats::Precision; use datafusion_common::{internal_err, not_impl_err, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; -use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ - equivalence::ProjectionMapping, + equivalence::{collapse_lex_req, ProjectionMapping}, expressions::{Column, Max, Min, UnKnownColumn}, - AggregateExpr, LexRequirement, PhysicalExpr, -}; -use datafusion_physical_expr::{ - physical_exprs_contains, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, + physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, + LexRequirement, PhysicalExpr, PhysicalSortRequirement, }; use itertools::Itertools; @@ -838,12 +835,10 @@ fn get_aggregate_expr_req( group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { - // If the aggregation function is ordering requirement is not absolutely necessary, or the aggregation - // is performing a "second stage" calculation, then ignore the ordering - // requirement. - if !aggr_expr.order_sensitivity().is_order_hard_required() - || !agg_mode.is_first_stage() - { + // If the aggregation function is ordering requirement is not absolutely + // necessary, or the aggregation is performing a "second stage" calculation, + // then ignore the ordering requirement. + if !aggr_expr.order_sensitivity().hard_requires() || !agg_mode.is_first_stage() { return vec![]; } From 9e50da625ffbb1084677b34a2580625054fa08fb Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 24 May 2024 15:46:06 +0300 Subject: [PATCH 74/77] Address reviews --- datafusion/functions-aggregate/src/first_last.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 20e49df0ad77..ca2caa14499c 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -159,7 +159,7 @@ impl AggregateUDFImpl for FirstValue { } fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { - datafusion_expr::ReversedUDAF::Reversed(Arc::new(LastValue::new())) + datafusion_expr::ReversedUDAF::Reversed(last_value_udaf().inner()) } } @@ -462,7 +462,7 @@ impl AggregateUDFImpl for LastValue { } fn reverse_expr(&self) -> datafusion_expr::ReversedUDAF { - datafusion_expr::ReversedUDAF::Reversed(Arc::new(FirstValue::new())) + datafusion_expr::ReversedUDAF::Reversed(first_value_udaf().inner()) } } From c9bcc89f794d98466afa94729d90daa64e6423aa Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 27 May 2024 09:35:04 +0300 Subject: [PATCH 75/77] Address reviews --- datafusion/functions-aggregate/src/first_last.rs | 14 +++++++++----- datafusion/physical-expr-common/src/sort_expr.rs | 9 ++++++--- datafusion/physical-expr-common/src/utils.rs | 14 +++++++++----- 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index ca2caa14499c..edccbac62e7f 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr_common::aggregate::utils::get_sort_options; use datafusion_physical_expr_common::sort_expr::{ - convert_logical_sort_exprs_to_physical, LexOrdering, PhysicalSortExpr, + limited_convert_logical_sort_exprs_to_physical, LexOrdering, PhysicalSortExpr, }; make_udaf_expr_and_func!( @@ -109,8 +109,10 @@ impl AggregateUDFImpl for FirstValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let ordering_req = - convert_logical_sort_exprs_to_physical(acc_args.sort_exprs, acc_args.schema)?; + let ordering_req = limited_convert_logical_sort_exprs_to_physical( + acc_args.sort_exprs, + acc_args.schema, + )?; let ordering_dtypes = ordering_req .iter() @@ -407,8 +409,10 @@ impl AggregateUDFImpl for LastValue { } fn accumulator(&self, acc_args: AccumulatorArgs) -> Result> { - let ordering_req = - convert_logical_sort_exprs_to_physical(acc_args.sort_exprs, acc_args.schema)?; + let ordering_req = limited_convert_logical_sort_exprs_to_physical( + acc_args.sort_exprs, + acc_args.schema, + )?; let ordering_dtypes = ordering_req .iter() diff --git a/datafusion/physical-expr-common/src/sort_expr.rs b/datafusion/physical-expr-common/src/sort_expr.rs index da55f9398087..f637355519af 100644 --- a/datafusion/physical-expr-common/src/sort_expr.rs +++ b/datafusion/physical-expr-common/src/sort_expr.rs @@ -22,7 +22,7 @@ use std::hash::{Hash, Hasher}; use std::sync::Arc; use crate::physical_expr::PhysicalExpr; -use crate::utils::convert_logical_expr_to_physical_expr; +use crate::utils::limited_convert_logical_expr_to_physical_expr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::datatypes::Schema; @@ -271,7 +271,7 @@ pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; /// Converts each [`Expr::Sort`] into a corresponding [`PhysicalSortExpr`]. /// Returns an error if the given logical expression is not a [`Expr::Sort`]. -pub fn convert_logical_sort_exprs_to_physical( +pub fn limited_convert_logical_sort_exprs_to_physical( exprs: &[Expr], schema: &Schema, ) -> Result> { @@ -282,7 +282,10 @@ pub fn convert_logical_sort_exprs_to_physical( return exec_err!("Expects to receive sort expression"); }; sort_exprs.push(PhysicalSortExpr { - expr: convert_logical_expr_to_physical_expr(sort.expr.as_ref(), schema)?, + expr: limited_convert_logical_expr_to_physical_expr( + sort.expr.as_ref(), + schema, + )?, options: SortOptions { descending: !sort.asc, nulls_first: sort.nulls_first, diff --git a/datafusion/physical-expr-common/src/utils.rs b/datafusion/physical-expr-common/src/utils.rs index 7e35534d67bb..cd6491e99322 100644 --- a/datafusion/physical-expr-common/src/utils.rs +++ b/datafusion/physical-expr-common/src/utils.rs @@ -105,20 +105,24 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec`. /// If conversion is not supported yet, returns Error. -pub fn convert_logical_expr_to_physical_expr( +pub fn limited_convert_logical_expr_to_physical_expr( expr: &Expr, schema: &Schema, ) -> Result> { match expr { Expr::Column(col) => expressions::column::col(&col.name, schema), Expr::Cast(cast_expr) => Ok(Arc::new(CastExpr::new( - convert_logical_expr_to_physical_expr(cast_expr.expr.as_ref(), schema)?, + limited_convert_logical_expr_to_physical_expr( + cast_expr.expr.as_ref(), + schema, + )?, cast_expr.data_type.clone(), None, ))), - Expr::Alias(alias_expr) => { - convert_logical_expr_to_physical_expr(alias_expr.expr.as_ref(), schema) - } + Expr::Alias(alias_expr) => limited_convert_logical_expr_to_physical_expr( + alias_expr.expr.as_ref(), + schema, + ), _ => exec_err!( "Unsupported expression: {expr} for conversion to Arc" ), From 93cccdb8b1bdf2a1fe0ccf27889c37e8c9419c4c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 27 May 2024 09:56:10 +0300 Subject: [PATCH 76/77] Update documentation, rename method --- .../physical_optimizer/update_aggr_exprs.rs | 6 ++-- datafusion/expr/src/udaf.rs | 35 ++++++++++++------- .../functions-aggregate/src/first_last.rs | 4 +-- .../physical-expr-common/src/aggregate/mod.rs | 6 ++-- 4 files changed, 30 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 11411e48b18d..2d27682c86b7 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -153,7 +153,7 @@ fn try_convert_aggregate_if_better( let reqs = concat_slices(prefix_requirement, &aggr_sort_reqs); if eq_properties.ordering_satisfy_requirement(&reqs) { // Existing ordering satisfies the aggregator requirements: - aggr_expr.with_requirement_satisfied(true)? + aggr_expr.has_beneficial_ordering(true)? } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &reverse_aggr_req, @@ -163,11 +163,11 @@ fn try_convert_aggregate_if_better( aggr_expr .reverse_expr() .unwrap_or(aggr_expr) - .with_requirement_satisfied(true)? + .has_beneficial_ordering(true)? } else { // There is no beneficial ordering present -- aggregation // will still work albeit in a less efficient mode. - aggr_expr.with_requirement_satisfied(false)? + aggr_expr.has_beneficial_ordering(false)? } .ok_or_else(|| { plan_datafusion_err!( diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index dff9cd88ec3a..541dd07c12d2 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -26,7 +26,7 @@ use crate::utils::AggregateOrderSensitivity; use crate::{Accumulator, Expr}; use crate::{AccumulatorFactoryFunction, ReturnTypeFunction, Signature}; use arrow::datatypes::{DataType, Field}; -use datafusion_common::{not_impl_err, Result}; +use datafusion_common::{exec_err, not_impl_err, Result}; use std::any::Any; use std::fmt::{self, Debug, Formatter}; use std::sync::Arc; @@ -194,16 +194,13 @@ impl AggregateUDF { self.inner.create_groups_accumulator() } - /// Sets the flag specifying whether the requirement of the UDF is satisfied. - /// A return value of `None` indicates that the UDF doesn't have support for - /// requirement satisfied mode (In this case, its requirement should be - /// definitely satisfied). - pub fn with_requirement_satisfied( + /// See [`AggregateUDFImpl::has_beneficial_ordering`] for more details. + pub fn has_beneficial_ordering( self, requirement_satisfied: bool, ) -> Result> { self.inner - .with_requirement_satisfied(requirement_satisfied) + .has_beneficial_ordering(requirement_satisfied) .map(|updated_udf| updated_udf.map(|udf| Self { inner: udf })) } @@ -392,15 +389,27 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { &[] } - /// Sets the flag specifying whether the requirement of the UDF is satisfied. - /// A return value of `None` indicates that the UDF doesn't have support for - /// requirement satisfied mode (In this case, its requirement should be - /// definitely satisfied). - fn with_requirement_satisfied( + /// Sets the indicator whether ordering requirements of the AggregateUDFImpl is + /// satisfied by its input. If this is not the case, UDFs with order + /// sensitivity `AggregateOrderSensitivity::Beneficial` can still produce + /// the correct result with possibly more work internally. + /// + /// # Returns + /// + /// Returns `Ok(Some(updated_udf))` if the process completes successfully. + /// If the expression can benefit from existing input ordering, but does + /// not implement the method, returns an error. Order insensitive and hard + /// requirement aggregators return `Ok(None)`. + fn has_beneficial_ordering( self: Arc, _requirement_satisfied: bool, ) -> Result>> { - // By default, no support for this optimization + if self.order_sensitivity().is_beneficial() { + return exec_err!( + "Should implement with satisfied for aggregator :{:?}", + self.name() + ); + } Ok(None) } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index edccbac62e7f..f449ac8c0a1a 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -147,7 +147,7 @@ impl AggregateUDFImpl for FirstValue { &self.aliases } - fn with_requirement_satisfied( + fn has_beneficial_ordering( self: Arc, requirement_satisfied: bool, ) -> Result>> { @@ -452,7 +452,7 @@ impl AggregateUDFImpl for LastValue { &self.aliases } - fn with_requirement_satisfied( + fn has_beneficial_ordering( self: Arc, requirement_satisfied: bool, ) -> Result>> { diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index ff783922e211..5cd61a8193e4 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -138,7 +138,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// If the expression can benefit from existing input ordering, but does /// not implement the method, returns an error. Order insensitive and hard /// requirement aggregators return `Ok(None)`. - fn with_requirement_satisfied( + fn has_beneficial_ordering( self: Arc, _requirement_satisfied: bool, ) -> Result>> { @@ -350,14 +350,14 @@ impl AggregateExpr for AggregateFunctionExpr { } } - fn with_requirement_satisfied( + fn has_beneficial_ordering( self: Arc, requirement_satisfied: bool, ) -> Result>> { let Some(updated_fn) = self .fun .clone() - .with_requirement_satisfied(requirement_satisfied)? + .has_beneficial_ordering(requirement_satisfied)? else { return Ok(None); }; From 4c430d31b39d70fa0b9ed8cef716eb91b71ee3aa Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 27 May 2024 13:13:00 +0300 Subject: [PATCH 77/77] Minor changes --- .../core/src/physical_optimizer/update_aggr_exprs.rs | 6 +++--- datafusion/expr/src/udaf.rs | 12 ++++++------ datafusion/functions-aggregate/src/first_last.rs | 12 ++++++------ datafusion/physical-expr-common/src/aggregate/mod.rs | 8 ++++---- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs index 2d27682c86b7..6a6ca815c510 100644 --- a/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs +++ b/datafusion/core/src/physical_optimizer/update_aggr_exprs.rs @@ -153,7 +153,7 @@ fn try_convert_aggregate_if_better( let reqs = concat_slices(prefix_requirement, &aggr_sort_reqs); if eq_properties.ordering_satisfy_requirement(&reqs) { // Existing ordering satisfies the aggregator requirements: - aggr_expr.has_beneficial_ordering(true)? + aggr_expr.with_beneficial_ordering(true)? } else if eq_properties.ordering_satisfy_requirement(&concat_slices( prefix_requirement, &reverse_aggr_req, @@ -163,11 +163,11 @@ fn try_convert_aggregate_if_better( aggr_expr .reverse_expr() .unwrap_or(aggr_expr) - .has_beneficial_ordering(true)? + .with_beneficial_ordering(true)? } else { // There is no beneficial ordering present -- aggregation // will still work albeit in a less efficient mode. - aggr_expr.has_beneficial_ordering(false)? + aggr_expr.with_beneficial_ordering(false)? } .ok_or_else(|| { plan_datafusion_err!( diff --git a/datafusion/expr/src/udaf.rs b/datafusion/expr/src/udaf.rs index 541dd07c12d2..0274038a36bf 100644 --- a/datafusion/expr/src/udaf.rs +++ b/datafusion/expr/src/udaf.rs @@ -194,13 +194,13 @@ impl AggregateUDF { self.inner.create_groups_accumulator() } - /// See [`AggregateUDFImpl::has_beneficial_ordering`] for more details. - pub fn has_beneficial_ordering( + /// See [`AggregateUDFImpl::with_beneficial_ordering`] for more details. + pub fn with_beneficial_ordering( self, - requirement_satisfied: bool, + beneficial_ordering: bool, ) -> Result> { self.inner - .has_beneficial_ordering(requirement_satisfied) + .with_beneficial_ordering(beneficial_ordering) .map(|updated_udf| updated_udf.map(|udf| Self { inner: udf })) } @@ -400,9 +400,9 @@ pub trait AggregateUDFImpl: Debug + Send + Sync { /// If the expression can benefit from existing input ordering, but does /// not implement the method, returns an error. Order insensitive and hard /// requirement aggregators return `Ok(None)`. - fn has_beneficial_ordering( + fn with_beneficial_ordering( self: Arc, - _requirement_satisfied: bool, + _beneficial_ordering: bool, ) -> Result>> { if self.order_sensitivity().is_beneficial() { return exec_err!( diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index f449ac8c0a1a..fd4e21971028 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -147,12 +147,12 @@ impl AggregateUDFImpl for FirstValue { &self.aliases } - fn has_beneficial_ordering( + fn with_beneficial_ordering( self: Arc, - requirement_satisfied: bool, + beneficial_ordering: bool, ) -> Result>> { Ok(Some(Arc::new( - FirstValue::new().with_requirement_satisfied(requirement_satisfied), + FirstValue::new().with_requirement_satisfied(beneficial_ordering), ))) } @@ -452,12 +452,12 @@ impl AggregateUDFImpl for LastValue { &self.aliases } - fn has_beneficial_ordering( + fn with_beneficial_ordering( self: Arc, - requirement_satisfied: bool, + beneficial_ordering: bool, ) -> Result>> { Ok(Some(Arc::new( - LastValue::new().with_requirement_satisfied(requirement_satisfied), + LastValue::new().with_requirement_satisfied(beneficial_ordering), ))) } diff --git a/datafusion/physical-expr-common/src/aggregate/mod.rs b/datafusion/physical-expr-common/src/aggregate/mod.rs index 5cd61a8193e4..503e2d8f9758 100644 --- a/datafusion/physical-expr-common/src/aggregate/mod.rs +++ b/datafusion/physical-expr-common/src/aggregate/mod.rs @@ -138,7 +138,7 @@ pub trait AggregateExpr: Send + Sync + Debug + PartialEq { /// If the expression can benefit from existing input ordering, but does /// not implement the method, returns an error. Order insensitive and hard /// requirement aggregators return `Ok(None)`. - fn has_beneficial_ordering( + fn with_beneficial_ordering( self: Arc, _requirement_satisfied: bool, ) -> Result>> { @@ -350,14 +350,14 @@ impl AggregateExpr for AggregateFunctionExpr { } } - fn has_beneficial_ordering( + fn with_beneficial_ordering( self: Arc, - requirement_satisfied: bool, + beneficial_ordering: bool, ) -> Result>> { let Some(updated_fn) = self .fun .clone() - .has_beneficial_ordering(requirement_satisfied)? + .with_beneficial_ordering(beneficial_ordering)? else { return Ok(None); };