From 8d8249e9b37b9ab0eafc6a2f470c4978a8597709 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 30 Nov 2023 14:42:34 +0300 Subject: [PATCH 01/49] Array agg ordered initial implementation --- datafusion/common/src/scalar.rs | 34 +++++ datafusion/common/src/utils.rs | 13 ++ .../src/physical_optimizer/enforce_sorting.rs | 141 ++++++++++++++++++ .../src/aggregate/array_agg_ordered.rs | 41 ++++- .../physical-plan/src/aggregates/mod.rs | 36 +++-- .../sqllogictest/test_files/groupby.slt | 82 +++++++++- 6 files changed, 324 insertions(+), 23 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 3431d71468ea..1f4f81eb5984 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -2112,6 +2112,40 @@ impl ScalarValue { Arc::new(array_into_large_list_array(values)) } + /// Converts `Vec` where each element has type corresponding to + /// `data_type`, to corresponding [`PrimitiveArray`]. + /// + /// Example + /// ``` + /// use std::sync::Arc; + /// use datafusion_common::ScalarValue; + /// use arrow::array::{ListArray, Int32Array}; + /// use arrow::datatypes::{DataType, Int32Type}; + /// use arrow_array::ArrayRef; + /// use datafusion_common::cast::as_list_array; + /// + /// let scalars = vec![ + /// ScalarValue::Int32(Some(1)), + /// ScalarValue::Int32(None), + /// ScalarValue::Int32(Some(2)) + /// ]; + /// + /// let array = ScalarValue::convert_to_arr(&scalars, &DataType::Int32); + /// + /// let expected = Arc::new(Int32Array::from( + /// vec![Some(1), None, Some(2)] + /// )) as ArrayRef; + /// + /// assert_eq!(&array, &expected); + /// ``` + pub fn convert_to_arr(values: &[ScalarValue], data_type: &DataType) -> ArrayRef { + if values.is_empty() { + new_empty_array(data_type) + } else { + Self::iter_to_array(values.iter().cloned()).unwrap() + } + } + /// Converts a scalar value into an array of `size` rows. /// /// # Errors diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index 12d4f516b4d0..f84c0956fcc3 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -291,6 +291,19 @@ pub fn get_arrayref_at_indices( .collect() } +/// Construct a new [`Vec`] of [`ArrayRef`] from the rows of the `arrays` at the `indices`. +pub fn get_elem_at_indices( + arrays: &[T], + indices: &PrimitiveArray, +) -> Vec { + (0..indices.len()) + .map(|idx| { + let idx = indices.value(idx) as usize; + arrays[idx].clone() + }) + .collect::>() +} + pub(crate) fn parse_identifiers_normalized(s: &str, ignore_case: bool) -> Vec { parse_identifiers(s) .unwrap_or_default() diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index ff052b5f040c..8a3a09c1b9a3 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2279,3 +2279,144 @@ mod tests { Ok(()) } } + +#[cfg(test)] +mod tmp_tests { + use crate::assert_batches_eq; + use crate::physical_plan::{collect, displayable, ExecutionPlan}; + use crate::prelude::SessionContext; + use arrow::util::pretty::print_batches; + use datafusion_common::Result; + use datafusion_execution::config::SessionConfig; + use datafusion_physical_plan::get_plan_string; + use std::sync::Arc; + + fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) + } + + #[tokio::test] + async fn test_query() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION 'tests/data/window_2.csv'", + ) + .await?; + + let sql = + "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM annotated_data_finite2 + GROUP BY a, b + ORDER BY a, b"; + + // let sql = "SELECT ARRAY_AGG(d ORDER BY c ASC) + // FROM annotated_data_finite2 + // GROUP BY a + // ORDER BY a"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query2() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(8) + .with_batch_size(4); + let ctx = SessionContext::new_with_config(config); + + ctx.sql( + "CREATE TABLE sales_global (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT + ) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), + (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0)", + ) + .await?; + + let sql = "SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 + FROM sales_global"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 9ca83a781a01..c8686383d6c3 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -31,9 +31,10 @@ use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; use arrow::datatypes::{DataType, Field}; use arrow_array::Array; +use arrow_ord::sort::{lexsort_to_indices, SortColumn}; use arrow_schema::{Fields, SortOptions}; use datafusion_common::cast::as_list_array; -use datafusion_common::utils::{compare_rows, get_row_at_idx}; +use datafusion_common::utils::{compare_rows, get_elem_at_indices, get_row_at_idx}; use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; @@ -263,7 +264,8 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } fn evaluate(&self) -> Result { - let arr = ScalarValue::new_list(&self.values, &self.datatypes[0]); + let (new_values, _new_ordering) = self.ordered_state()?; + let arr = ScalarValue::new_list(&new_values, &self.datatypes[0]); Ok(ScalarValue::List(arr)) } @@ -318,10 +320,10 @@ impl OrderSensitiveArrayAggAccumulator { } fn evaluate_orderings(&self) -> Result { + let (_new_values, new_ordering_values) = self.ordered_state()?; let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); let struct_field = Fields::from(fields.clone()); - let orderings: Vec = self - .ordering_values + let orderings: Vec = new_ordering_values .iter() .map(|ordering| { ScalarValue::Struct(Some(ordering.clone()), struct_field.clone()) @@ -332,6 +334,37 @@ impl OrderSensitiveArrayAggAccumulator { let arr = ScalarValue::new_list(&orderings, &struct_type); Ok(ScalarValue::List(arr)) } + + fn ordered_state(&self) -> Result<(Vec, Vec>)> { + let n_col = self.ordering_req.len(); + let n_row = self.ordering_values.len(); + + // Transpose + let orderings = (0..n_col) + .map(|col_idx| { + (0..n_row) + .map(|row_idx| self.ordering_values[row_idx][col_idx].clone()) + .collect::>() + }) + .collect::>(); + + // Sort data according to requirements + let sort_columns = orderings + .into_iter() + .zip(self.ordering_req.iter()) + .map(|(ordering, sort_expr)| { + let values = ScalarValue::convert_to_arr(&ordering, &self.datatypes[0]); + SortColumn { + values, + options: Some(sort_expr.options), + } + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, None)?; + let new_values = get_elem_at_indices(&self.values, &indices); + let new_ordering_values = get_elem_at_indices(&self.ordering_values, &indices); + Ok((new_values, new_ordering_values)) + } } /// This is a wrapper struct to be able to correctly merge ARRAY_AGG diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7d7fba6ef6c3..3e37505307b6 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -385,12 +385,13 @@ fn get_finest_requirement( continue; } } - // If neither of the requirements 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" - ); + // // If neither of the requirements 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" + // ); + return Ok(None); } else { finest_req = Some(fn_req.clone()); } @@ -997,14 +998,21 @@ fn aggregate_expressions( // In partial mode, append ordering requirements to expressions' results. // Ordering requirements are used by subsequent executors to satisfy the required // ordering for `AggregateMode::FinalPartitioned`/`AggregateMode::Final` modes. - if matches!(mode, AggregateMode::Partial) { - if let Some(ordering_req) = agg.order_bys() { - let ordering_exprs = ordering_req - .iter() - .map(|item| item.expr.clone()) - .collect::>(); - result.extend(ordering_exprs); - } + // if matches!(mode, AggregateMode::Partial) { + // if let Some(ordering_req) = agg.order_bys() { + // let ordering_exprs = ordering_req + // .iter() + // .map(|item| item.expr.clone()) + // .collect::>(); + // result.extend(ordering_exprs); + // } + // } + if let Some(ordering_req) = agg.order_bys() { + let ordering_exprs = ordering_req + .iter() + .map(|item| item.expr.clone()) + .collect::>(); + result.extend(ordering_exprs); } result }) diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 1d6d7dc671fa..ffb325f3ad52 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2339,15 +2339,37 @@ FRA [200.0, 50.0] 250 TUR [100.0, 75.0] 175 GRC [80.0, 30.0] 110 -# test_ordering_sensitive_aggregation3 -# When different aggregators have conflicting requirements, we cannot satisfy all of them in current implementation. -# test below should raise Plan Error. -statement error DataFusion error: This feature is not implemented: Conflicting ordering requirements in aggregate functions is not supported -SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, +query TT +EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, + ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, + ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 + FROM sales_global AS s + GROUP BY s.country + ORDER BY s.country +---- +logical_plan +Sort: s.country ASC NULLS LAST +--Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] AS amounts, ARRAY_AGG(s.amount) ORDER BY [s.amount ASC NULLS LAST] AS amounts2, ARRAY_AGG(s.amount) ORDER BY [s.sn ASC NULLS LAST] AS amounts3 +----Aggregate: groupBy=[[s.country]], aggr=[[ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST], ARRAY_AGG(s.amount) ORDER BY [s.amount ASC NULLS LAST], ARRAY_AGG(s.amount) ORDER BY [s.sn ASC NULLS LAST]]] +------SubqueryAlias: s +--------TableScan: sales_global projection=[country, sn, amount] +physical_plan +SortExec: expr=[country@0 ASC NULLS LAST] +--ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, ARRAY_AGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]@2 as amounts2, ARRAY_AGG(s.amount) ORDER BY [s.sn ASC NULLS LAST]@3 as amounts3] +----AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), ARRAY_AGG(s.amount), ARRAY_AGG(s.amount)] +------MemoryExec: partitions=1, partition_sizes=[1] + +query T??? +SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 FROM sales_global AS s GROUP BY s.country + ORDER BY s.country +---- +FRA [200.0, 50.0] [50.0, 200.0] [50.0, 200.0] +GRC [80.0, 30.0] [30.0, 80.0] [30.0, 80.0] +TUR [100.0, 75.0] [75.0, 100.0] [75.0, 100.0] # test_ordering_sensitive_aggregation4 # If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should append requirement to @@ -4047,3 +4069,53 @@ set datafusion.sql_parser.dialect = 'Generic'; statement ok drop table aggregate_test_100; + +statement ok +set datafusion.optimizer.prefer_existing_sort = true; + +query TT +EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +logical_plan +Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +--Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST], ARRAY_AGG(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.d DESC NULLS FIRST]]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----CoalesceBatchesExec: target_batch_size=2 +------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST +--------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + +statement ok +set datafusion.execution.target_partitions = 1; + +query II?? +SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +statement ok +set datafusion.execution.target_partitions = 8; + +query II?? +SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] From 8f54037ba7ed8981c52d27409c02a2bd74d246bc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 30 Nov 2023 14:43:06 +0300 Subject: [PATCH 02/49] Minor changes --- datafusion/sqllogictest/test_files/groupby.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index ffb325f3ad52..01ee7a8ae44c 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4090,7 +4090,7 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; From 7484c109f6ddb6323de756cbd08a0c0e6d30bc06 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 30 Nov 2023 17:37:26 +0300 Subject: [PATCH 03/49] Add tests for first_value, last_value --- .../src/physical_optimizer/enforce_sorting.rs | 59 +++++++ .../physical-expr/src/aggregate/first_last.rs | 5 + .../sqllogictest/test_files/groupby.slt | 150 ++++++++++++++++++ 3 files changed, 214 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 8a3a09c1b9a3..3164556d634d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2419,4 +2419,63 @@ mod tmp_tests { assert_batches_eq!(expected, &batches); Ok(()) } + + #[tokio::test] + async fn test_query3() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(8) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION 'tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } } diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 0dc27dede8b6..85ed3a1852e7 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -204,6 +204,11 @@ impl Accumulator for FirstValueAccumulator { fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it if !values[0].is_empty() && !self.is_set { + println!("----------"); + for value in values{ + println!("value: {:?}", value); + } + println!("----------"); let row = get_row_at_idx(values, 0)?; // Update with first value in the array. self.update_with_new_row(&row); diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 01ee7a8ae44c..6991afbddbbb 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4119,3 +4119,153 @@ SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) 0 1 [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] 1 2 [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +query TT +EXPLAIN SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +logical_plan +Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +--Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[FIRST_VALUE(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST], LAST_VALUE(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST], ARRAY_AGG(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.d DESC NULLS FIRST]]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----CoalesceBatchesExec: target_batch_size=2 +------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST +--------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true + +statement ok +set datafusion.execution.target_partitions = 1; + +query IIII? +SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 4 0 [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 3 0 [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +statement ok +set datafusion.execution.target_partitions = 8; + +query IIII? +SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 4 0 [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 3 0 [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +# Test same queries where there is sort inserted +statement ok +set datafusion.optimizer.prefer_existing_sort = false; + +statement ok +set datafusion.execution.batch_size = 20; + +query TT +EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +logical_plan +Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +--Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[ARRAY_AGG(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST], ARRAY_AGG(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.d DESC NULLS FIRST]]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +------CoalesceBatchesExec: target_batch_size=20 +--------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 +----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true + +statement ok +set datafusion.execution.target_partitions = 1; + +query II?? +SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +statement ok +set datafusion.execution.target_partitions = 8; + +query II?? +SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +query TT +EXPLAIN SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +logical_plan +Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULLS LAST +--Aggregate: groupBy=[[multiple_ordered_table.a, multiple_ordered_table.b]], aggr=[[FIRST_VALUE(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST], LAST_VALUE(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST], ARRAY_AGG(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.d DESC NULLS FIRST]]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] +------CoalesceBatchesExec: target_batch_size=20 +--------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 +----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true + +statement ok +set datafusion.execution.target_partitions = 1; + +query IIII? +SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 4 0 [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 3 0 [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +statement ok +set datafusion.execution.target_partitions = 8; + +query IIII? +SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 4 0 [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] +0 1 3 0 [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] +1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] +1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] From 071b2660f10d6b6417fd77a82850c1b0302096dd Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 14:39:09 +0300 Subject: [PATCH 04/49] Add new tests --- .../src/physical_optimizer/enforce_sorting.rs | 59 +++++++++++++++++++ .../physical-expr/src/aggregate/first_last.rs | 25 +++++--- .../sqllogictest/test_files/groupby.slt | 11 ++++ 3 files changed, 87 insertions(+), 8 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 3164556d634d..20c4eb13b8aa 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2478,4 +2478,63 @@ mod tmp_tests { assert_batches_eq!(expected, &batches); Ok(()) } + + #[tokio::test] + async fn test_query4() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(8) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION 'tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } } diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 85ed3a1852e7..62ba513cd44a 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -203,16 +203,25 @@ impl Accumulator for FirstValueAccumulator { fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it - if !values[0].is_empty() && !self.is_set { - println!("----------"); - for value in values{ - println!("value: {:?}", value); - } - println!("----------"); - let row = get_row_at_idx(values, 0)?; - // Update with first value in the array. + if !values[0].is_empty() { + let sort_columns = values[1..].iter().zip(self.ordering_req.iter()).map(|(col, sort_expr)| { + SortColumn { values: col.clone(), options: Some(sort_expr.options) } + }).collect::>(); + let indices = lexsort_to_indices(&sort_columns, Some(1))?; + let first_idx = indices.value(0) as usize; + let row = get_row_at_idx(values, first_idx)?; self.update_with_new_row(&row); } + // if !values[0].is_empty() && !self.is_set { + // println!("----------"); + // for value in values{ + // println!("value: {:?}", value); + // } + // println!("----------"); + // let row = get_row_at_idx(values, 0)?; + // // Update with first value in the array. + // self.update_with_new_row(&row); + // } Ok(()) } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 6991afbddbbb..81cec13efd42 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4256,6 +4256,17 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRA 1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] +query IIII +SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 4 0 +0 1 3 0 +1 2 4 0 +1 3 3 0 + statement ok set datafusion.execution.target_partitions = 8; From 6b77b404616aa3363923abd80b1c375a7ff7f83a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 15:59:36 +0300 Subject: [PATCH 05/49] Add first, last tests --- .../src/physical_optimizer/enforce_sorting.rs | 610 ++++++++++-------- .../physical-expr/src/aggregate/first_last.rs | 115 +++- .../physical-plan/src/aggregates/mod.rs | 2 +- .../sqllogictest/test_files/groupby.slt | 18 +- 4 files changed, 448 insertions(+), 297 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 20c4eb13b8aa..e25437e7dc34 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2280,261 +2280,355 @@ mod tests { } } -#[cfg(test)] -mod tmp_tests { - use crate::assert_batches_eq; - use crate::physical_plan::{collect, displayable, ExecutionPlan}; - use crate::prelude::SessionContext; - use arrow::util::pretty::print_batches; - use datafusion_common::Result; - use datafusion_execution::config::SessionConfig; - use datafusion_physical_plan::get_plan_string; - use std::sync::Arc; - - fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) - } - - #[tokio::test] - async fn test_query() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::new_with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION 'tests/data/window_2.csv'", - ) - .await?; - - let sql = - "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) - FROM annotated_data_finite2 - GROUP BY a, b - ORDER BY a, b"; - - // let sql = "SELECT ARRAY_AGG(d ORDER BY c ASC) - // FROM annotated_data_finite2 - // GROUP BY a - // ORDER BY a"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query2() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(8) - .with_batch_size(4); - let ctx = SessionContext::new_with_config(config); - - ctx.sql( - "CREATE TABLE sales_global (zip_code INT, - country VARCHAR(3), - sn INT, - ts TIMESTAMP, - currency VARCHAR(3), - amount FLOAT - ) as VALUES - (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), - (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), - (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), - (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), - (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), - (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0)", - ) - .await?; - - let sql = "SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 - FROM sales_global"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query3() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(8) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE multiple_ordered_table ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION 'tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) - FROM multiple_ordered_table - GROUP BY a, b - ORDER BY a, b"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query4() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(8) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE multiple_ordered_table ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION 'tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) - FROM multiple_ordered_table - GROUP BY a, b - ORDER BY a, b"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } -} +// #[cfg(test)] +// mod tmp_tests { +// use crate::assert_batches_eq; +// use crate::physical_plan::{collect, displayable, ExecutionPlan}; +// use crate::prelude::SessionContext; +// use arrow::util::pretty::print_batches; +// use datafusion_common::Result; +// use datafusion_execution::config::SessionConfig; +// use datafusion_physical_plan::get_plan_string; +// use std::sync::Arc; +// +// fn print_plan(plan: &Arc) -> Result<()> { +// let formatted = displayable(plan.as_ref()).indent(true).to_string(); +// let actual: Vec<&str> = formatted.trim().lines().collect(); +// println!("{:#?}", actual); +// Ok(()) +// } +// +// const MULTIPLE_ORDERED_TABLE: &str = "CREATE EXTERNAL TABLE multiple_ordered_table ( +// a0 INTEGER, +// a INTEGER, +// b INTEGER, +// c INTEGER, +// d INTEGER +// ) +// STORED AS CSV +// WITH HEADER ROW +// WITH ORDER (a ASC, b ASC) +// WITH ORDER (c ASC) +// LOCATION 'tests/data/window_2.csv'"; +// const ANNOTATED_DATA_FINITE2: &str = "CREATE EXTERNAL TABLE annotated_data_finite2 ( +// a0 INTEGER, +// a INTEGER, +// b INTEGER, +// c INTEGER, +// d INTEGER +// ) +// STORED AS CSV +// WITH HEADER ROW +// WITH ORDER (a ASC, b ASC, c ASC) +// LOCATION 'tests/data/window_2.csv'"; +// +// const SALES_GLOBAL: &str = "CREATE TABLE sales_global (zip_code INT, +// country VARCHAR(3), +// sn INT, +// ts TIMESTAMP, +// currency VARCHAR(3), +// amount FLOAT +// ) as VALUES +// (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), +// (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), +// (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), +// (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), +// (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), +// (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0)"; +// +// #[tokio::test] +// async fn test_query() -> Result<()> { +// let config = SessionConfig::new().with_target_partitions(1); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// ANNOTATED_DATA_FINITE2, +// ) +// .await?; +// +// let sql = +// "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) +// FROM annotated_data_finite2 +// GROUP BY a, b +// ORDER BY a, b"; +// +// // let sql = "SELECT ARRAY_AGG(d ORDER BY c ASC) +// // FROM annotated_data_finite2 +// // GROUP BY a +// // ORDER BY a"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query2() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(8) +// .with_batch_size(4); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// SALES_GLOBAL, +// ) +// .await?; +// +// let sql = "SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 +// FROM sales_global"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query3() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(8) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// MULTIPLE_ORDERED_TABLE, +// ) +// .await?; +// +// let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) +// FROM multiple_ordered_table +// GROUP BY a, b +// ORDER BY a, b"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query4() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(8) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// MULTIPLE_ORDERED_TABLE, +// ) +// .await?; +// +// let sql = +// "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) +// FROM multiple_ordered_table +// GROUP BY a, b +// ORDER BY a, b"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query5() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(1) +// .with_batch_size(4); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// MULTIPLE_ORDERED_TABLE, +// ) +// .await?; +// +// let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) +// FROM multiple_ordered_table +// GROUP BY a, b +// ORDER BY a, b"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query6() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(8) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql( +// SALES_GLOBAL, +// ) +// .await?; +// +// let sql = "SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, +// LAST_VALUE(amount ORDER BY ts ASC) AS fv2 +// FROM sales_global"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// } diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 62ba513cd44a..000d67e319b2 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -186,10 +186,26 @@ impl FirstValueAccumulator { } // Updates state with the values in the given row. - fn update_with_new_row(&mut self, row: &[ScalarValue]) { - self.first = row[0].clone(); - self.orderings = row[1..].to_vec(); - self.is_set = true; + fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { + if !self.is_set { + self.first = row[0].clone(); + self.orderings = row[1..].to_vec(); + self.is_set = true; + } else if compare_rows( + &self.orderings, + &row[1..], + &self + .ordering_req + .iter() + .map(|sort_expr| sort_expr.options) + .collect::>(), + )? + .is_gt() + { + self.first = row[0].clone(); + self.orderings = row[1..].to_vec(); + } + Ok(()) } } @@ -204,24 +220,10 @@ impl Accumulator for FirstValueAccumulator { fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it if !values[0].is_empty() { - let sort_columns = values[1..].iter().zip(self.ordering_req.iter()).map(|(col, sort_expr)| { - SortColumn { values: col.clone(), options: Some(sort_expr.options) } - }).collect::>(); - let indices = lexsort_to_indices(&sort_columns, Some(1))?; - let first_idx = indices.value(0) as usize; + let first_idx = self.get_first_value_idx(values)?; let row = get_row_at_idx(values, first_idx)?; - self.update_with_new_row(&row); + self.update_with_new_row(&row)?; } - // if !values[0].is_empty() && !self.is_set { - // println!("----------"); - // for value in values{ - // println!("value: {:?}", value); - // } - // println!("----------"); - // let row = get_row_at_idx(values, 0)?; - // // Update with first value in the array. - // self.update_with_new_row(&row); - // } Ok(()) } @@ -254,7 +256,7 @@ impl Accumulator for FirstValueAccumulator { // Update with first value in the state. Note that we should exclude the // is_set flag from the state. Otherwise, we will end up with a state // containing two is_set flags. - self.update_with_new_row(&first_row[0..is_set_idx]); + self.update_with_new_row(&first_row[0..is_set_idx])?; } } Ok(()) @@ -272,6 +274,25 @@ impl Accumulator for FirstValueAccumulator { } } +impl FirstValueAccumulator { + fn get_first_value_idx(&self, values: &[ArrayRef]) -> Result { + Ok(if self.ordering_req.is_empty() { + values[0].len() - 1 + } else { + let sort_columns = values[1..] + .iter() + .zip(self.ordering_req.iter()) + .map(|(col, sort_expr)| SortColumn { + values: col.clone(), + options: Some(sort_expr.options), + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, Some(1))?; + indices.value(0) as usize + }) + } +} + /// LAST_VALUE aggregate expression #[derive(Debug)] pub struct LastValue { @@ -423,10 +444,26 @@ impl LastValueAccumulator { } // Updates state with the values in the given row. - fn update_with_new_row(&mut self, row: &[ScalarValue]) { - self.last = row[0].clone(); - self.orderings = row[1..].to_vec(); - self.is_set = true; + fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { + if !self.is_set { + self.last = row[0].clone(); + self.orderings = row[1..].to_vec(); + self.is_set = true; + } else if compare_rows( + &self.orderings, + &row[1..], + &self + .ordering_req + .iter() + .map(|sort_expr| sort_expr.options) + .collect::>(), + )? + .is_lt() + { + self.last = row[0].clone(); + self.orderings = row[1..].to_vec(); + } + Ok(()) } } @@ -439,10 +476,11 @@ impl Accumulator for LastValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { + // If we have seen first value, we shouldn't update it if !values[0].is_empty() { - let row = get_row_at_idx(values, values[0].len() - 1)?; - // Update with last value in the array. - self.update_with_new_row(&row); + let last_idx = self.get_last_value_idx(values)?; + let row = get_row_at_idx(values, last_idx)?; + self.update_with_new_row(&row)?; } Ok(()) } @@ -479,7 +517,7 @@ impl Accumulator for LastValueAccumulator { // Update with last value in the state. Note that we should exclude the // is_set flag from the state. Otherwise, we will end up with a state // containing two is_set flags. - self.update_with_new_row(&last_row[0..is_set_idx]); + self.update_with_new_row(&last_row[0..is_set_idx])?; } } Ok(()) @@ -497,6 +535,25 @@ impl Accumulator for LastValueAccumulator { } } +impl LastValueAccumulator { + fn get_last_value_idx(&self, values: &[ArrayRef]) -> Result { + Ok(if self.ordering_req.is_empty() { + values[0].len() - 1 + } else { + let sort_columns = values[1..] + .iter() + .zip(self.ordering_req.iter()) + .map(|(col, sort_expr)| SortColumn { + values: col.clone(), + options: Some(!sort_expr.options), + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, Some(1))?; + indices.value(0) as usize + }) + } +} + /// Filters states according to the `is_set` flag at the last column and returns /// the resulting states. fn filter_states_according_to_is_set( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3e37505307b6..890ea3462163 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -39,7 +39,7 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; +use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 81cec13efd42..c97b9264f6d9 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2212,10 +2212,10 @@ SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c FROM annotated_data_infinite2 GROUP BY a, b ---- -0 0 24 -0 1 49 -1 2 74 -1 3 99 +0 0 0 +0 1 25 +1 2 50 +1 3 75 # when LAST_VALUE, or FIRST_VALUE value do not contain ordering requirement # queries should still work, However, result depends on the scanning order and @@ -2261,7 +2261,7 @@ CREATE TABLE sales_global (zip_code INT, (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), - (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) + (0, 'GRC', 4, '2022-01-03 11:00:00'::timestamp, 'EUR', 80.0) # create a new table named exchange rates statement ok @@ -2703,8 +2703,8 @@ SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, FROM sales_global GROUP BY country ---- -TUR 100 75 175 GRC 80 30 110 +TUR 100 75 175 FRA 200 50 250 query TT @@ -2750,7 +2750,7 @@ ORDER BY s.sn 1 FRA 1 2022-01-01T08:00:00 EUR 50 1 TUR 2 2022-01-01T11:30:00 TRY 75 1 FRA 3 2022-01-02T12:00:00 EUR 200 -0 GRC 4 2022-01-03T10:00:00 EUR 80 +0 GRC 4 2022-01-03T11:00:00 EUR 80 1 TUR 4 2022-01-03T10:00:00 TRY 100 # Run order-sensitive aggregators in multiple partitions @@ -2936,7 +2936,7 @@ query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 FROM sales_global ---- -[100.0, 80.0, 200.0, 75.0, 50.0, 30.0] +[80.0, 100.0, 200.0, 75.0, 50.0, 30.0] # ARRAY_AGG should work in multiple partitions query TT @@ -3080,7 +3080,7 @@ ORDER BY s.sn 1 FRA 1 2022-01-01T08:00:00 EUR 50 1 TUR 2 2022-01-01T11:30:00 TRY 75 1 FRA 3 2022-01-02T12:00:00 EUR 200 -0 GRC 4 2022-01-03T10:00:00 EUR 80 +0 GRC 4 2022-01-03T11:00:00 EUR 80 1 TUR 4 2022-01-03T10:00:00 TRY 100 # create a table for testing From 49d5f279ab70eb0072ccb5ec96e5eaad4c63d144 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 16:16:25 +0300 Subject: [PATCH 06/49] Minor changes --- .../physical-expr/src/aggregate/first_last.rs | 76 +++++++++---------- 1 file changed, 36 insertions(+), 40 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 000d67e319b2..623f8a355938 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -220,7 +220,7 @@ impl Accumulator for FirstValueAccumulator { fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it if !values[0].is_empty() { - let first_idx = self.get_first_value_idx(values)?; + let first_idx = get_value_idx::(values, &self.ordering_req)?; let row = get_row_at_idx(values, first_idx)?; self.update_with_new_row(&row)?; } @@ -274,25 +274,6 @@ impl Accumulator for FirstValueAccumulator { } } -impl FirstValueAccumulator { - fn get_first_value_idx(&self, values: &[ArrayRef]) -> Result { - Ok(if self.ordering_req.is_empty() { - values[0].len() - 1 - } else { - let sort_columns = values[1..] - .iter() - .zip(self.ordering_req.iter()) - .map(|(col, sort_expr)| SortColumn { - values: col.clone(), - options: Some(sort_expr.options), - }) - .collect::>(); - let indices = lexsort_to_indices(&sort_columns, Some(1))?; - indices.value(0) as usize - }) - } -} - /// LAST_VALUE aggregate expression #[derive(Debug)] pub struct LastValue { @@ -478,7 +459,7 @@ impl Accumulator for LastValueAccumulator { fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { // If we have seen first value, we shouldn't update it if !values[0].is_empty() { - let last_idx = self.get_last_value_idx(values)?; + let last_idx = get_value_idx::(values, &self.ordering_req)?; let row = get_row_at_idx(values, last_idx)?; self.update_with_new_row(&row)?; } @@ -535,25 +516,6 @@ impl Accumulator for LastValueAccumulator { } } -impl LastValueAccumulator { - fn get_last_value_idx(&self, values: &[ArrayRef]) -> Result { - Ok(if self.ordering_req.is_empty() { - values[0].len() - 1 - } else { - let sort_columns = values[1..] - .iter() - .zip(self.ordering_req.iter()) - .map(|(col, sort_expr)| SortColumn { - values: col.clone(), - options: Some(!sort_expr.options), - }) - .collect::>(); - let indices = lexsort_to_indices(&sort_columns, Some(1))?; - indices.value(0) as usize - }) - } -} - /// Filters states according to the `is_set` flag at the last column and returns /// the resulting states. fn filter_states_according_to_is_set( @@ -588,6 +550,40 @@ fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { .collect::>() } +/// Gets either first, or last value index inside values columns according to ordering requirements +fn get_value_idx( + values: &[ArrayRef], + ordering_req: &[PhysicalSortExpr], +) -> Result { + let value = &values[0]; + let ordering_values = &values[1..]; + Ok(if ordering_req.is_empty() { + value.len() - 1 + } else { + let sort_options = if FIRST { + get_sort_options(ordering_req) + } else { + // last + // Reverse requirement options (e.g last, is the first entry in the reverse order) + ordering_req + .iter() + .map(|sort_expr| !sort_expr.options) + .collect() + }; + + let sort_columns = ordering_values + .iter() + .zip(sort_options) + .map(|(col, options)| SortColumn { + values: col.clone(), + options: Some(options), + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, Some(1))?; + indices.value(0) as usize + }) +} + #[cfg(test)] mod tests { use crate::aggregate::first_last::{FirstValueAccumulator, LastValueAccumulator}; From 7704b9788bfd38a7cd698c06cfba898f04b21c70 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 16:19:25 +0300 Subject: [PATCH 07/49] Simplifications --- .../physical-expr/src/aggregate/first_last.rs | 32 ++++++++----------- 1 file changed, 14 insertions(+), 18 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 623f8a355938..7f7509e300a0 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -187,23 +187,19 @@ impl FirstValueAccumulator { // Updates state with the values in the given row. fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { - if !self.is_set { - self.first = row[0].clone(); - self.orderings = row[1..].to_vec(); - self.is_set = true; - } else if compare_rows( - &self.orderings, - &row[1..], - &self - .ordering_req - .iter() - .map(|sort_expr| sort_expr.options) - .collect::>(), - )? - .is_gt() + let value = &row[0]; + let orderings = &row[1..]; + if !self.is_set + || compare_rows( + &self.orderings, + &orderings, + &get_sort_options(&self.ordering_req), + )? + .is_gt() { - self.first = row[0].clone(); - self.orderings = row[1..].to_vec(); + self.first = value.clone(); + self.orderings = orderings.to_vec(); + self.is_set = true; } Ok(()) } @@ -218,7 +214,7 @@ impl Accumulator for FirstValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - // If we have seen first value, we shouldn't update it + // For empty batches there is nothing to update if !values[0].is_empty() { let first_idx = get_value_idx::(values, &self.ordering_req)?; let row = get_row_at_idx(values, first_idx)?; @@ -457,7 +453,7 @@ impl Accumulator for LastValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - // If we have seen first value, we shouldn't update it + // For empty batches there is nothing to update if !values[0].is_empty() { let last_idx = get_value_idx::(values, &self.ordering_req)?; let row = get_row_at_idx(values, last_idx)?; From b803a6f88819adf55207014ce01b06bcbb00f28b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 16:21:14 +0300 Subject: [PATCH 08/49] Minor changes --- .../physical-expr/src/aggregate/first_last.rs | 28 ++++++++----------- 1 file changed, 12 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 7f7509e300a0..06d379a1a6f6 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -422,23 +422,19 @@ impl LastValueAccumulator { // Updates state with the values in the given row. fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { - if !self.is_set { - self.last = row[0].clone(); - self.orderings = row[1..].to_vec(); - self.is_set = true; - } else if compare_rows( - &self.orderings, - &row[1..], - &self - .ordering_req - .iter() - .map(|sort_expr| sort_expr.options) - .collect::>(), - )? - .is_lt() + let value = &row[0]; + let orderings = &row[1..]; + if !self.is_set + || compare_rows( + &self.orderings, + &orderings, + &get_sort_options(&self.ordering_req), + )? + .is_lt() { - self.last = row[0].clone(); - self.orderings = row[1..].to_vec(); + self.last = value.clone(); + self.orderings = orderings.to_vec(); + self.is_set = true; } Ok(()) } From 5913e0cb72f704c438587e2bfcb33fb814e8a72a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 17:19:54 +0300 Subject: [PATCH 09/49] Minor changes --- .../physical-expr/src/aggregate/first_last.rs | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 06d379a1a6f6..526dbfd5b292 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -214,9 +214,9 @@ impl Accumulator for FirstValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - // For empty batches there is nothing to update - if !values[0].is_empty() { - let first_idx = get_value_idx::(values, &self.ordering_req)?; + if let Some(first_idx) = + get_value_idx::(values, &self.ordering_req, self.is_set)? + { let row = get_row_at_idx(values, first_idx)?; self.update_with_new_row(&row)?; } @@ -449,9 +449,9 @@ impl Accumulator for LastValueAccumulator { } fn update_batch(&mut self, values: &[ArrayRef]) -> Result<()> { - // For empty batches there is nothing to update - if !values[0].is_empty() { - let last_idx = get_value_idx::(values, &self.ordering_req)?; + if let Some(last_idx) = + get_value_idx::(values, &self.ordering_req, self.is_set)? + { let row = get_row_at_idx(values, last_idx)?; self.update_with_new_row(&row)?; } @@ -546,12 +546,25 @@ fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { fn get_value_idx( values: &[ArrayRef], ordering_req: &[PhysicalSortExpr], -) -> Result { + is_set: bool, +) -> Result> { let value = &values[0]; let ordering_values = &values[1..]; - Ok(if ordering_req.is_empty() { + if value.is_empty() { + // For empty batches there is nothing to update + return Ok(None); + } + + Ok(Some(if ordering_req.is_empty() && !is_set && FIRST { + 0 + } else if ordering_req.is_empty() && is_set && FIRST { + // No need to overwrite existing value, when no ordering is specified + // Just use first value encountered + return Ok(None); + } else if ordering_req.is_empty() && !FIRST { value.len() - 1 } else { + // Calculate real first, or last value according to requirement. let sort_options = if FIRST { get_sort_options(ordering_req) } else { @@ -573,7 +586,7 @@ fn get_value_idx( .collect::>(); let indices = lexsort_to_indices(&sort_columns, Some(1))?; indices.value(0) as usize - }) + })) } #[cfg(test)] From 4a4f1f82ce6c2f22216b8d1c52b7e4fe1d631889 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 17:24:46 +0300 Subject: [PATCH 10/49] Minor changes --- datafusion/physical-expr/src/aggregate/first_last.rs | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 526dbfd5b292..82999ba62add 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -189,10 +189,13 @@ impl FirstValueAccumulator { fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { let value = &row[0]; let orderings = &row[1..]; + // Update when + // - no entry in the state + // - There is an earlier entry in according to requirements if !self.is_set || compare_rows( &self.orderings, - &orderings, + orderings, &get_sort_options(&self.ordering_req), )? .is_gt() @@ -424,10 +427,15 @@ impl LastValueAccumulator { fn update_with_new_row(&mut self, row: &[ScalarValue]) -> Result<()> { let value = &row[0]; let orderings = &row[1..]; + // Update when + // - no value in the state + // - There is no specific requirement, but a new value (most recent entry in terms of execution) + // - There is a more recent entry in terms of requirement if !self.is_set + || self.orderings.is_empty() || compare_rows( &self.orderings, - &orderings, + orderings, &get_sort_options(&self.ordering_req), )? .is_lt() From 18815ea5d67225aa15aaa33b99419c4a07b0cec2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 17:55:39 +0300 Subject: [PATCH 11/49] Resolve linter errors --- datafusion/common/src/utils.rs | 13 -- .../src/aggregate/array_agg_ordered.rs | 116 +++++++++++------- 2 files changed, 72 insertions(+), 57 deletions(-) diff --git a/datafusion/common/src/utils.rs b/datafusion/common/src/utils.rs index f84c0956fcc3..12d4f516b4d0 100644 --- a/datafusion/common/src/utils.rs +++ b/datafusion/common/src/utils.rs @@ -291,19 +291,6 @@ pub fn get_arrayref_at_indices( .collect() } -/// Construct a new [`Vec`] of [`ArrayRef`] from the rows of the `arrays` at the `indices`. -pub fn get_elem_at_indices( - arrays: &[T], - indices: &PrimitiveArray, -) -> Vec { - (0..indices.len()) - .map(|idx| { - let idx = indices.value(idx) as usize; - arrays[idx].clone() - }) - .collect::>() -} - pub(crate) fn parse_identifiers_normalized(s: &str, ignore_case: bool) -> Vec { parse_identifiers(s) .unwrap_or_default() diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index c8686383d6c3..7836a8d21b38 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -29,12 +29,13 @@ use crate::expressions::format_state_name; use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; +use arrow::compute; use arrow::datatypes::{DataType, Field}; use arrow_array::Array; use arrow_ord::sort::{lexsort_to_indices, SortColumn}; use arrow_schema::{Fields, SortOptions}; use datafusion_common::cast::as_list_array; -use datafusion_common::utils::{compare_rows, get_elem_at_indices, get_row_at_idx}; +use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; @@ -194,13 +195,17 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { return Ok(()); } - let n_row = values[0].len(); - for index in 0..n_row { - let row = get_row_at_idx(values, index)?; - self.values.push(row[0].clone()); - self.ordering_values.push(row[1..].to_vec()); - } + let (new_values, new_ordering_values) = self.reorder_according_to_reqs(values)?; + let sort_options = get_sort_options(&self.ordering_req); + // Merge new values and new orderings + let (merged_values, merged_ordering_values) = merge_ordered_arrays( + &[self.values.clone(), new_values], + &[self.ordering_values.clone(), new_ordering_values], + &sort_options, + )?; + self.values = merged_values; + self.ordering_values = merged_ordering_values; Ok(()) } @@ -239,11 +244,7 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { partition_ordering_values.push(v); } - let sort_options = self - .ordering_req - .iter() - .map(|sort_expr| sort_expr.options) - .collect::>(); + let sort_options = get_sort_options(&self.ordering_req); let (new_values, new_orderings) = merge_ordered_arrays( &partition_values, &partition_ordering_values, @@ -264,8 +265,7 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } fn evaluate(&self) -> Result { - let (new_values, _new_ordering) = self.ordered_state()?; - let arr = ScalarValue::new_list(&new_values, &self.datatypes[0]); + let arr = ScalarValue::new_list(&self.values, &self.datatypes[0]); Ok(ScalarValue::List(arr)) } @@ -320,10 +320,10 @@ impl OrderSensitiveArrayAggAccumulator { } fn evaluate_orderings(&self) -> Result { - let (_new_values, new_ordering_values) = self.ordered_state()?; let fields = ordering_fields(&self.ordering_req, &self.datatypes[1..]); let struct_field = Fields::from(fields.clone()); - let orderings: Vec = new_ordering_values + let orderings: Vec = self + .ordering_values .iter() .map(|ordering| { ScalarValue::Struct(Some(ordering.clone()), struct_field.clone()) @@ -335,35 +335,55 @@ impl OrderSensitiveArrayAggAccumulator { Ok(ScalarValue::List(arr)) } - fn ordered_state(&self) -> Result<(Vec, Vec>)> { - let n_col = self.ordering_req.len(); - let n_row = self.ordering_values.len(); + fn reorder_according_to_reqs( + &self, + values: &[ArrayRef], + ) -> Result<(Vec, Vec>)> { + let value = &values[0]; + let orderings = &values[1..]; - // Transpose - let orderings = (0..n_col) - .map(|col_idx| { - (0..n_row) - .map(|row_idx| self.ordering_values[row_idx][col_idx].clone()) - .collect::>() - }) - .collect::>(); - - // Sort data according to requirements - let sort_columns = orderings - .into_iter() - .zip(self.ordering_req.iter()) - .map(|(ordering, sort_expr)| { - let values = ScalarValue::convert_to_arr(&ordering, &self.datatypes[0]); - SortColumn { - values, - options: Some(sort_expr.options), - } - }) - .collect::>(); - let indices = lexsort_to_indices(&sort_columns, None)?; - let new_values = get_elem_at_indices(&self.values, &indices); - let new_ordering_values = get_elem_at_indices(&self.ordering_values, &indices); - Ok((new_values, new_ordering_values)) + if self.ordering_req.is_empty() { + // No requirement + Self::convert_arr_to_vec(value, orderings) + } else { + let sort_options = get_sort_options(&self.ordering_req); + // Sort data according to requirements + let sort_columns = orderings + .iter() + .zip(sort_options) + .map(|(ordering, options)| SortColumn { + values: ordering.clone(), + options: Some(options), + }) + .collect::>(); + let indices = lexsort_to_indices(&sort_columns, None)?; + let sorted_value = compute::take( + value.as_ref(), + &indices, + None, // None: no index check + )?; + let orderings = get_arrayref_at_indices(orderings, &indices)?; + + Self::convert_arr_to_vec(&sorted_value, &orderings) + } + } + + fn convert_arr_to_vec( + value: &ArrayRef, + orderings: &[ArrayRef], + ) -> Result<(Vec, Vec>)> { + let n_row = value.len(); + // Convert &[ArrayRef] to Vec> + let orderings = (0..n_row) + .map(|idx| get_row_at_idx(orderings, idx)) + .collect::>>()?; + + // Convert ArrayRef to Vec + let value = (0..n_row) + .map(|idx| ScalarValue::try_from_array(value, idx)) + .collect::>>()?; + + Ok((value, orderings)) } } @@ -542,6 +562,14 @@ fn merge_ordered_arrays( Ok((merged_values, merged_orderings)) } +/// Selects the sort option attribute from all the given `PhysicalSortExpr`s. +fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { + ordering_req + .iter() + .map(|item| item.options) + .collect::>() +} + #[cfg(test)] mod tests { use crate::aggregate::array_agg_ordered::merge_ordered_arrays; From edf5c5c433839cb239c167743b461a7e06281abb Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 18:01:30 +0300 Subject: [PATCH 12/49] Remove clone --- .../src/aggregate/array_agg_ordered.rs | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 7836a8d21b38..68f93ec75b50 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -200,8 +200,8 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { // Merge new values and new orderings let (merged_values, merged_ordering_values) = merge_ordered_arrays( - &[self.values.clone(), new_values], - &[self.ordering_values.clone(), new_ordering_values], + &[&self.values, &new_values], + &[&self.ordering_values, &new_ordering_values], &sort_options, )?; self.values = merged_values; @@ -227,20 +227,20 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { let mut partition_ordering_values = vec![]; // Existing values should be merged also. - partition_values.push(self.values.clone()); - partition_ordering_values.push(self.ordering_values.clone()); + partition_values.push(self.values.as_slice()); + partition_ordering_values.push(self.ordering_values.as_slice()); let array_agg_res = ScalarValue::convert_array_to_scalar_vec(array_agg_values)?; - for v in array_agg_res.into_iter() { + for v in array_agg_res.iter() { partition_values.push(v); } let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)?; // Ordering requirement expression values for each entry in the ARRAY_AGG list let other_ordering_values = self.convert_array_agg_to_orderings(orderings)?; - for v in other_ordering_values.into_iter() { + for v in other_ordering_values.iter() { partition_ordering_values.push(v); } @@ -477,11 +477,11 @@ impl<'a> PartialOrd for CustomElement<'a> { /// Inner `Vec`s of the `ordering_values` will be compared according `sort_options` (Their sizes should match) fn merge_ordered_arrays( // We will merge values into single `Vec`. - values: &[Vec], + values: &[&[ScalarValue]], // `values` will be merged according to `ordering_values`. // Inner `Vec` can be thought as ordering information for the // each `ScalarValue` in the values`. - ordering_values: &[Vec>], + ordering_values: &[&[Vec]], // Defines according to which ordering comparisons should be done. sort_options: &[SortOptions], ) -> Result<(Vec, Vec>)> { @@ -626,8 +626,8 @@ mod tests { ]; let (merged_vals, merged_ts) = merge_ordered_arrays( - &[lhs_vals, rhs_vals], - &[lhs_orderings, rhs_orderings], + &[&lhs_vals, &rhs_vals], + &[&lhs_orderings, &rhs_orderings], &sort_options, )?; let merged_vals = ScalarValue::iter_to_array(merged_vals.into_iter())?; @@ -693,8 +693,8 @@ mod tests { Arc::new(Int64Array::from(vec![4, 4, 3, 3, 2, 2, 1, 1, 0, 0])) as ArrayRef, ]; let (merged_vals, merged_ts) = merge_ordered_arrays( - &[lhs_vals, rhs_vals], - &[lhs_orderings, rhs_orderings], + &[&lhs_vals, &rhs_vals], + &[&lhs_orderings, &rhs_orderings], &sort_options, )?; let merged_vals = ScalarValue::iter_to_array(merged_vals.into_iter())?; From ea6d94abe8603f4440d7064576b34a855952fdb1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 1 Dec 2023 18:03:15 +0300 Subject: [PATCH 13/49] Remove unused code --- datafusion/common/src/scalar.rs | 34 --------------------------------- 1 file changed, 34 deletions(-) diff --git a/datafusion/common/src/scalar.rs b/datafusion/common/src/scalar.rs index 1f4f81eb5984..3431d71468ea 100644 --- a/datafusion/common/src/scalar.rs +++ b/datafusion/common/src/scalar.rs @@ -2112,40 +2112,6 @@ impl ScalarValue { Arc::new(array_into_large_list_array(values)) } - /// Converts `Vec` where each element has type corresponding to - /// `data_type`, to corresponding [`PrimitiveArray`]. - /// - /// Example - /// ``` - /// use std::sync::Arc; - /// use datafusion_common::ScalarValue; - /// use arrow::array::{ListArray, Int32Array}; - /// use arrow::datatypes::{DataType, Int32Type}; - /// use arrow_array::ArrayRef; - /// use datafusion_common::cast::as_list_array; - /// - /// let scalars = vec![ - /// ScalarValue::Int32(Some(1)), - /// ScalarValue::Int32(None), - /// ScalarValue::Int32(Some(2)) - /// ]; - /// - /// let array = ScalarValue::convert_to_arr(&scalars, &DataType::Int32); - /// - /// let expected = Arc::new(Int32Array::from( - /// vec![Some(1), None, Some(2)] - /// )) as ArrayRef; - /// - /// assert_eq!(&array, &expected); - /// ``` - pub fn convert_to_arr(values: &[ScalarValue], data_type: &DataType) -> ArrayRef { - if values.is_empty() { - new_empty_array(data_type) - } else { - Self::iter_to_array(values.iter().cloned()).unwrap() - } - } - /// Converts a scalar value into an array of `size` rows. /// /// # Errors From ffbe3bc57cb429af0caef71c2f086f742fb05fa9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 4 Dec 2023 14:14:52 +0300 Subject: [PATCH 14/49] Add grouping support --- .../src/physical_optimizer/enforce_sorting.rs | 1049 +++++++++++------ .../src/aggregate/array_agg_ordered.rs | 43 +- .../physical-expr/src/aggregate/first_last.rs | 89 +- .../physical-plan/src/aggregates/mod.rs | 134 +++ .../src/aggregates/no_grouping.rs | 145 ++- .../physical-plan/src/aggregates/row_hash.rs | 357 ++++-- .../sqllogictest/test_files/groupby.slt | 163 ++- 7 files changed, 1291 insertions(+), 689 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index e25437e7dc34..be6b6853c73f 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2280,355 +2280,700 @@ mod tests { } } -// #[cfg(test)] -// mod tmp_tests { -// use crate::assert_batches_eq; -// use crate::physical_plan::{collect, displayable, ExecutionPlan}; -// use crate::prelude::SessionContext; -// use arrow::util::pretty::print_batches; -// use datafusion_common::Result; -// use datafusion_execution::config::SessionConfig; -// use datafusion_physical_plan::get_plan_string; -// use std::sync::Arc; -// -// fn print_plan(plan: &Arc) -> Result<()> { -// let formatted = displayable(plan.as_ref()).indent(true).to_string(); -// let actual: Vec<&str> = formatted.trim().lines().collect(); -// println!("{:#?}", actual); -// Ok(()) -// } -// -// const MULTIPLE_ORDERED_TABLE: &str = "CREATE EXTERNAL TABLE multiple_ordered_table ( -// a0 INTEGER, -// a INTEGER, -// b INTEGER, -// c INTEGER, -// d INTEGER -// ) -// STORED AS CSV -// WITH HEADER ROW -// WITH ORDER (a ASC, b ASC) -// WITH ORDER (c ASC) -// LOCATION 'tests/data/window_2.csv'"; -// const ANNOTATED_DATA_FINITE2: &str = "CREATE EXTERNAL TABLE annotated_data_finite2 ( -// a0 INTEGER, -// a INTEGER, -// b INTEGER, -// c INTEGER, -// d INTEGER -// ) -// STORED AS CSV -// WITH HEADER ROW -// WITH ORDER (a ASC, b ASC, c ASC) -// LOCATION 'tests/data/window_2.csv'"; -// -// const SALES_GLOBAL: &str = "CREATE TABLE sales_global (zip_code INT, -// country VARCHAR(3), -// sn INT, -// ts TIMESTAMP, -// currency VARCHAR(3), -// amount FLOAT -// ) as VALUES -// (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), -// (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), -// (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), -// (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), -// (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), -// (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0)"; -// -// #[tokio::test] -// async fn test_query() -> Result<()> { -// let config = SessionConfig::new().with_target_partitions(1); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// ANNOTATED_DATA_FINITE2, -// ) -// .await?; -// -// let sql = -// "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) -// FROM annotated_data_finite2 -// GROUP BY a, b -// ORDER BY a, b"; -// -// // let sql = "SELECT ARRAY_AGG(d ORDER BY c ASC) -// // FROM annotated_data_finite2 -// // GROUP BY a -// // ORDER BY a"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query2() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(8) -// .with_batch_size(4); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// SALES_GLOBAL, -// ) -// .await?; -// -// let sql = "SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 -// FROM sales_global"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query3() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(8) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// MULTIPLE_ORDERED_TABLE, -// ) -// .await?; -// -// let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) -// FROM multiple_ordered_table -// GROUP BY a, b -// ORDER BY a, b"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query4() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(8) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// MULTIPLE_ORDERED_TABLE, -// ) -// .await?; -// -// let sql = -// "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) -// FROM multiple_ordered_table -// GROUP BY a, b -// ORDER BY a, b"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query5() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(1) -// .with_batch_size(4); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// MULTIPLE_ORDERED_TABLE, -// ) -// .await?; -// -// let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) -// FROM multiple_ordered_table -// GROUP BY a, b -// ORDER BY a, b"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query6() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(8) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql( -// SALES_GLOBAL, -// ) -// .await?; -// -// let sql = "SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, -// LAST_VALUE(amount ORDER BY ts ASC) AS fv2 -// FROM sales_global"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// } +#[cfg(test)] +mod tmp_tests { + use crate::assert_batches_eq; + use crate::physical_plan::{collect, displayable, ExecutionPlan}; + use crate::prelude::SessionContext; + use arrow::util::pretty::print_batches; + use datafusion_common::Result; + use datafusion_execution::config::SessionConfig; + use datafusion_physical_plan::get_plan_string; + use std::sync::Arc; + + fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) + } + + const MULTIPLE_ORDERED_TABLE: &str = "CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) + LOCATION 'tests/data/window_2.csv'"; + const ANNOTATED_DATA_FINITE2: &str = "CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION 'tests/data/window_2.csv'"; + + const SALES_GLOBAL: &str = "CREATE TABLE sales_global (zip_code INT, + country VARCHAR(3), + sn INT, + ts TIMESTAMP, + currency VARCHAR(3), + amount FLOAT + ) as VALUES + (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), + (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), + (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), + (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), + (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), + (0, 'GRC', 4, '2022-01-03 11:00:00'::timestamp, 'EUR', 80.0)"; + + const T1: &str = + "CREATE TABLE tab1(col0 INTEGER, col1 INTEGER, col2 INTEGER) as VALUES + (22,6,8), + (28,57,45), + (82,44,71) + "; + + const ANNOTATED_DATA_INFINITE2: &str = + "CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC, c ASC) +LOCATION '../core/tests/data/window_2.csv';"; + + // #[tokio::test] + // async fn test_query() -> Result<()> { + // let config = SessionConfig::new().with_target_partitions(1); + // let ctx = SessionContext::new_with_config(config); + // + // ctx.sql(ANNOTATED_DATA_FINITE2).await?; + // + // let sql = + // "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + // FROM annotated_data_finite2 + // GROUP BY a, b + // ORDER BY a, b"; + // + // // let sql = "SELECT ARRAY_AGG(d ORDER BY c ASC) + // // FROM annotated_data_finite2 + // // GROUP BY a + // // ORDER BY a"; + // + // let msg = format!("Creating logical plan for '{sql}'"); + // let dataframe = ctx.sql(sql).await.expect(&msg); + // let physical_plan = dataframe.create_physical_plan().await?; + // print_plan(&physical_plan)?; + // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&batches)?; + // + // let expected = vec![ + // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } + // + // #[tokio::test] + // async fn test_query2() -> Result<()> { + // let config = SessionConfig::new() + // .with_target_partitions(8) + // .with_batch_size(4); + // let ctx = SessionContext::new_with_config(config); + // + // ctx.sql(SALES_GLOBAL).await?; + // + // let sql = "SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 + // FROM sales_global"; + // + // let msg = format!("Creating logical plan for '{sql}'"); + // let dataframe = ctx.sql(sql).await.expect(&msg); + // let physical_plan = dataframe.create_physical_plan().await?; + // print_plan(&physical_plan)?; + // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&batches)?; + // + // let expected = vec![ + // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } + // + // #[tokio::test] + // async fn test_query3() -> Result<()> { + // let config = SessionConfig::new() + // .with_target_partitions(8) + // .with_batch_size(1000); + // let ctx = SessionContext::new_with_config(config); + // + // ctx.sql(MULTIPLE_ORDERED_TABLE).await?; + // + // let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + // FROM multiple_ordered_table + // GROUP BY a, b + // ORDER BY a, b"; + // + // let msg = format!("Creating logical plan for '{sql}'"); + // let dataframe = ctx.sql(sql).await.expect(&msg); + // let physical_plan = dataframe.create_physical_plan().await?; + // print_plan(&physical_plan)?; + // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&batches)?; + // + // let expected = vec![ + // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } + // + // #[tokio::test] + // async fn test_query4() -> Result<()> { + // let config = SessionConfig::new() + // .with_target_partitions(8) + // .with_batch_size(1000); + // let ctx = SessionContext::new_with_config(config); + // + // ctx.sql(MULTIPLE_ORDERED_TABLE).await?; + // + // let sql = + // "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) + // FROM multiple_ordered_table + // GROUP BY a, b + // ORDER BY a, b"; + // + // let msg = format!("Creating logical plan for '{sql}'"); + // let dataframe = ctx.sql(sql).await.expect(&msg); + // let physical_plan = dataframe.create_physical_plan().await?; + // print_plan(&physical_plan)?; + // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&batches)?; + // + // let expected = vec![ + // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } + // + // #[tokio::test] + // async fn test_query5() -> Result<()> { + // let config = SessionConfig::new() + // .with_target_partitions(1) + // .with_batch_size(4); + // let ctx = SessionContext::new_with_config(config); + // + // ctx.sql(MULTIPLE_ORDERED_TABLE).await?; + // + // let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + // FROM multiple_ordered_table + // GROUP BY a, b + // ORDER BY a, b"; + // + // let msg = format!("Creating logical plan for '{sql}'"); + // let dataframe = ctx.sql(sql).await.expect(&msg); + // let physical_plan = dataframe.create_physical_plan().await?; + // print_plan(&physical_plan)?; + // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&batches)?; + // + // let expected = vec![ + // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } + // + // #[tokio::test] + // async fn test_query6() -> Result<()> { + // let config = SessionConfig::new() + // .with_target_partitions(8) + // .with_batch_size(1000); + // let ctx = SessionContext::new_with_config(config); + // + // ctx.sql(SALES_GLOBAL).await?; + // + // let sql = "SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, + // LAST_VALUE(amount ORDER BY ts ASC) AS fv2 + // FROM sales_global"; + // + // let msg = format!("Creating logical plan for '{sql}'"); + // let dataframe = ctx.sql(sql).await.expect(&msg); + // let physical_plan = dataframe.create_physical_plan().await?; + // print_plan(&physical_plan)?; + // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&batches)?; + // + // let expected = vec![ + // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", + // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + // ]; + // // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + // + // let expected = [ + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + // ]; + // assert_batches_eq!(expected, &batches); + // Ok(()) + // } + + #[tokio::test] + async fn test_query7() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(1) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(SALES_GLOBAL).await?; + + let sql = "SELECT + FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, + LAST_VALUE(amount ORDER BY ts ASC) AS lv1, + FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, + LAST_VALUE(amount ORDER BY ts DESC) AS lv2 + FROM sales_global"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as lv2]", + " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+------+------+------+------+", + "| fv1 | lv1 | fv2 | lv2 |", + "+------+------+------+------+", + "| 30.0 | 80.0 | 80.0 | 30.0 |", + "+------+------+------+------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query8() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(1) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(SALES_GLOBAL).await?; + + let sql = "SELECT + FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, + LAST_VALUE(amount ORDER BY ts ASC) AS lv1, + FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, + LAST_VALUE(amount ORDER BY ts DESC) AS lv2 + FROM sales_global + GROUP BY country + ORDER BY country"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "ProjectionExec: expr=[fv1@0 as fv1, lv1@1 as lv1, fv2@2 as fv2, lv2@3 as lv2]", + " SortExec: expr=[country@4 ASC NULLS LAST]", + " ProjectionExec: expr=[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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@4 as lv2, country@0 as country]", + " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+------+-------+-------+------+", + "| fv1 | lv1 | fv2 | lv2 |", + "+------+-------+-------+------+", + "| 50.0 | 200.0 | 200.0 | 50.0 |", + "| 30.0 | 80.0 | 80.0 | 30.0 |", + "| 75.0 | 100.0 | 100.0 | 75.0 |", + "+------+-------+-------+------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query9() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(1) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(T1).await?; + + let sql = "SELECT - tab1.col0 * 84 + + 38 AS col2 FROM tab1 GROUP BY tab1.col0"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "ProjectionExec: expr=[CAST((- col0@0) AS Int64) * 84 + 38 as col2]", + " AggregateExec: mode=Single, gby=[col0@0 as col0], aggr=[]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+-------+", + "| col2 |", + "+-------+", + "| -1810 |", + "| -2314 |", + "| -6850 |", + "+-------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query10() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(1) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(ANNOTATED_DATA_INFINITE2).await?; + + let sql = "SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c + FROM annotated_data_infinite2 + GROUP BY a, b"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c]", + " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST]", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---+---------+", + "| a | b | first_c |", + "+---+---+---------+", + "| 0 | 0 | 0 |", + "| 0 | 1 | 25 |", + "| 1 | 2 | 50 |", + "| 1 | 3 | 75 |", + "+---+---+---------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query11() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(1) + .with_batch_size(1000); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(MULTIPLE_ORDERED_TABLE).await?; + + let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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=Single, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.c), LAST_VALUE(multiple_ordered_table.c)]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---------+--------+", + "| d | first_a | last_c |", + "+---+---------+--------+", + "| 3 | 99 | 15 |", + "| 4 | 98 | 9 |", + "| 2 | 97 | 1 |", + "| 0 | 95 | 0 |", + "| 1 | 90 | 4 |", + "+---+---------+--------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query12() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(8) + .with_batch_size(2); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(MULTIPLE_ORDERED_TABLE).await?; + + let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d +ORDER BY d"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "SortPreservingMergeExec: [d@0 ASC NULLS LAST]", + " SortExec: expr=[d@0 ASC NULLS LAST]", + " ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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.c), 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.c), LAST_VALUE(multiple_ordered_table.c)]", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---------+--------+", + "| d | first_a | last_c |", + "+---+---------+--------+", + "| 0 | 95 | 0 |", + "| 1 | 90 | 4 |", + "| 2 | 97 | 1 |", + "| 3 | 99 | 15 |", + "| 4 | 98 | 9 |", + "+---+---------+--------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } + + #[tokio::test] + async fn test_query13() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(8) + .with_batch_size(2); + let ctx = SessionContext::new_with_config(config); + + ctx.sql(MULTIPLE_ORDERED_TABLE).await?; + + let sql = + "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&batches)?; + + let expected = vec![ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", + " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", + " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=2", + " RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8", + " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected = [ + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| a | b | ARRAY_AGG(multiple_ordered_table.d) | ARRAY_AGG(multiple_ordered_table.d) |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", + "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", + "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", + "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", + "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", + ]; + assert_batches_eq!(expected, &batches); + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 68f93ec75b50..b8c461f42443 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -29,13 +29,11 @@ use crate::expressions::format_state_name; use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; use arrow::array::ArrayRef; -use arrow::compute; use arrow::datatypes::{DataType, Field}; use arrow_array::Array; -use arrow_ord::sort::{lexsort_to_indices, SortColumn}; use arrow_schema::{Fields, SortOptions}; use datafusion_common::cast::as_list_array; -use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; +use datafusion_common::utils::{compare_rows, get_row_at_idx}; use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; @@ -194,8 +192,10 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { if values.is_empty() { return Ok(()); } - - let (new_values, new_ordering_values) = self.reorder_according_to_reqs(values)?; + let value = &values[0]; + let orderings = &values[1..]; + let (new_values, new_ordering_values) = + Self::convert_arr_to_vec(value, orderings)?; let sort_options = get_sort_options(&self.ordering_req); // Merge new values and new orderings @@ -335,39 +335,6 @@ impl OrderSensitiveArrayAggAccumulator { Ok(ScalarValue::List(arr)) } - fn reorder_according_to_reqs( - &self, - values: &[ArrayRef], - ) -> Result<(Vec, Vec>)> { - let value = &values[0]; - let orderings = &values[1..]; - - if self.ordering_req.is_empty() { - // No requirement - Self::convert_arr_to_vec(value, orderings) - } else { - let sort_options = get_sort_options(&self.ordering_req); - // Sort data according to requirements - let sort_columns = orderings - .iter() - .zip(sort_options) - .map(|(ordering, options)| SortColumn { - values: ordering.clone(), - options: Some(options), - }) - .collect::>(); - let indices = lexsort_to_indices(&sort_columns, None)?; - let sorted_value = compute::take( - value.as_ref(), - &indices, - None, // None: no index check - )?; - let orderings = get_arrayref_at_indices(orderings, &indices)?; - - Self::convert_arr_to_vec(&sorted_value, &orderings) - } - } - fn convert_arr_to_vec( value: &ArrayRef, orderings: &[ArrayRef], diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 82999ba62add..c79bcfc1e376 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -23,7 +23,8 @@ use std::sync::Arc; use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; use crate::expressions::format_state_name; use crate::{ - reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, + reverse_order_bys, AggregateExpr, LexOrdering, LexOrderingRef, PhysicalExpr, + PhysicalSortExpr, }; use arrow::array::{Array, ArrayRef, AsArray, BooleanArray}; @@ -550,50 +551,76 @@ fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { .collect::>() } +// /// Gets either first, or last value index inside values columns according to ordering requirements +// fn get_value_idx( +// values: &[ArrayRef], +// ordering_req: &[PhysicalSortExpr], +// is_set: bool, +// ) -> Result> { +// let value = &values[0]; +// let ordering_values = &values[1..]; +// if value.is_empty() { +// // For empty batches there is nothing to update +// return Ok(None); +// } +// +// Ok(Some(if ordering_req.is_empty() && !is_set && FIRST { +// 0 +// } else if ordering_req.is_empty() && is_set && FIRST { +// // No need to overwrite existing value, when no ordering is specified +// // Just use first value encountered +// return Ok(None); +// } else if ordering_req.is_empty() && !FIRST { +// value.len() - 1 +// } else { +// // Calculate real first, or last value according to requirement. +// let sort_options = if FIRST { +// get_sort_options(ordering_req) +// } else { +// // last +// // Reverse requirement options (e.g last, is the first entry in the reverse order) +// ordering_req +// .iter() +// .map(|sort_expr| !sort_expr.options) +// .collect() +// }; +// +// let sort_columns = ordering_values +// .iter() +// .zip(sort_options) +// .map(|(col, options)| SortColumn { +// values: col.clone(), +// options: Some(options), +// }) +// .collect::>(); +// let indices = lexsort_to_indices(&sort_columns, Some(1))?; +// indices.value(0) as usize +// })) +// } + /// Gets either first, or last value index inside values columns according to ordering requirements fn get_value_idx( values: &[ArrayRef], - ordering_req: &[PhysicalSortExpr], + ordering_req: LexOrderingRef, is_set: bool, ) -> Result> { let value = &values[0]; - let ordering_values = &values[1..]; + let _ordering_values = &values[1..]; if value.is_empty() { // For empty batches there is nothing to update return Ok(None); } - - Ok(Some(if ordering_req.is_empty() && !is_set && FIRST { - 0 - } else if ordering_req.is_empty() && is_set && FIRST { + if is_set && FIRST && ordering_req.is_empty() { // No need to overwrite existing value, when no ordering is specified // Just use first value encountered return Ok(None); - } else if ordering_req.is_empty() && !FIRST { - value.len() - 1 - } else { - // Calculate real first, or last value according to requirement. - let sort_options = if FIRST { - get_sort_options(ordering_req) - } else { - // last - // Reverse requirement options (e.g last, is the first entry in the reverse order) - ordering_req - .iter() - .map(|sort_expr| !sort_expr.options) - .collect() - }; + } - let sort_columns = ordering_values - .iter() - .zip(sort_options) - .map(|(col, options)| SortColumn { - values: col.clone(), - options: Some(options), - }) - .collect::>(); - let indices = lexsort_to_indices(&sort_columns, Some(1))?; - indices.value(0) as usize + Ok(Some(if FIRST { + 0 + } else { + // LAST + value.len() - 1 })) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 890ea3462163..bca74b9b05b9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -49,6 +49,7 @@ use datafusion_physical_expr::{ physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; +use futures::StreamExt; use itertools::{izip, Itertools}; @@ -269,6 +270,23 @@ impl From for SendableRecordBatchStream { } } +// pub struct AggregateGroup{ +// /// Aggregate expressions +// aggr_expr: Vec>, +// /// FILTER (WHERE clause) expression for each aggregate expression +// filter_expr: Vec>>, +// /// (ORDER BY clause) expression for each aggregate expression +// /// TODO: Make below variable LexOrdering +// order_by_expr: Vec>, +// } + +pub struct AggregateGroup { + aggregate_expressions: Vec>>, + filter_expressions: Vec>>, + accumulators: Vec, + requirement: LexOrdering, +} + /// Hash aggregate execution plan #[derive(Debug)] pub struct AggregateExec { @@ -335,6 +353,7 @@ fn get_finest_requirement( order_by_expr: &mut [Option], eq_properties: &EquivalenceProperties, ) -> Result> { + return Ok(None); // First, we check if all the requirements are satisfied by the existing // ordering. If so, we return `None` to indicate this. let mut all_satisfied = true; @@ -979,6 +998,28 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } +fn get_groups_indices( + aggr_exprs: &[Arc], + group_by: &PhysicalGroupBy, +) -> Vec<(Vec, LexOrdering)> { + let mut initial_groups = vec![]; + for idx in 0..aggr_exprs.len() { + let aggr_expr = &aggr_exprs[idx]; + let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + if !is_order_sensitive(aggr_expr) + || group_by_contains_all_requirements(&group_by, &req) + { + // No requirement. + req.clear(); + } + // let req = aggr_expr[idx].order_bys().unwrap_or_default().to_vec(); + initial_groups.push((vec![idx], req)); + } + // TODO: Add merge groups logic. + + initial_groups +} + /// returns physical expressions for arguments to evaluate against a batch /// The expressions are different depending on `mode`: /// * Partial: AggregateExpr::expressions @@ -1061,6 +1102,27 @@ fn create_accumulators( .collect::>>() } +/// returns a vector of ArrayRefs, where each entry corresponds to either the +/// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) +fn finalize_aggregation_groups( + // accumulators: &[AccumulatorItem], + aggregate_groups: &[AggregateGroup], + // TODO: Use Mapping indices + mode: &AggregateMode, +) -> Result> { + let elems = aggregate_groups + .iter() + .map(|aggregate_group| finalize_aggregation(&aggregate_group.accumulators, mode)) + .collect::>>()?; + // TODO: Add proper indices + // Convert Vec> to Vec. + let res = elems + .into_iter() + .flat_map(|elems| elems) + .collect::>(); + Ok(res) +} + /// returns a vector of ArrayRefs, where each entry corresponds to either the /// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) fn finalize_aggregation( @@ -1239,6 +1301,29 @@ mod tests { Ok(schema) } + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect::>() + } + + // Convert each inner tuple to PhysicalSortExpr + fn convert_to_orderings( + orderings: &[Vec<(&Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) + .collect() + } + /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. @@ -2163,4 +2248,53 @@ mod tests { assert_eq!(res, common_requirement); Ok(()) } + + #[tokio::test] + async fn test_calc_aggregate_groups() -> 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 option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // This is the reverse requirement of options1 + let options_dess = SortOptions { + descending: true, + nulls_first: true, + }; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + + let test_cases = vec![ + // ------- TEST CASE 1 ----------- + ( + // Ordering requirements + vec![vec![(col_a, option_asc)]], + // expected + vec![vec![0]], + ), + ]; + for (ordering_reqs, expected) in test_cases { + let aggr_exprs = ordering_reqs + .into_iter() + .map(|req| { + let req = convert_to_sort_exprs(&req); + let aggr_expr = Arc::new(FirstValue::new( + col_a.clone(), + "first1", + DataType::Int32, + req, + vec![], + )) as _; + aggr_expr + }) + .collect::>(); + + let res = get_groups_indices(&aggr_exprs); + assert_eq!(res, expected); + } + Ok(()) + } } diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 90eb488a2ead..8a0b3b9a4cba 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -18,7 +18,8 @@ //! Aggregate without grouping columns use crate::aggregates::{ - aggregate_expressions, create_accumulators, finalize_aggregation, AccumulatorItem, + aggregate_expressions, create_accumulators, finalize_aggregation, + finalize_aggregation_groups, get_groups_indices, AccumulatorItem, AggregateGroup, AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; @@ -34,6 +35,8 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::filter::batch_filter; +use crate::sorts::sort::sort_batch; +use datafusion_common::utils::get_at_indices; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use futures::stream::{Stream, StreamExt}; @@ -57,13 +60,20 @@ struct AggregateStreamInner { mode: AggregateMode, input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, - aggregate_expressions: Vec>>, - filter_expressions: Vec>>, - accumulators: Vec, + // aggregate_expressions: Vec>>, + // filter_expressions: Vec>>, + // accumulators: Vec, + aggregate_groups: Vec, reservation: MemoryReservation, finished: bool, } +// struct AggregateGroup{ +// aggregate_expressions: Vec>>, +// filter_expressions: Vec>>, +// accumulators: Vec, +// } + impl AggregateStream { /// Create a new AggregateStream pub fn new( @@ -76,7 +86,7 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - + let group_indices = get_groups_indices(&agg.aggr_expr, agg.group_by()); let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { AggregateMode::Partial @@ -90,15 +100,34 @@ impl AggregateStream { let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); - + let aggregate_groups = group_indices + .into_iter() + .map(|(indices, requirement)| { + let aggr_exprs = get_at_indices(&agg.aggr_expr, &indices)?; + let aggregate_expressions = + get_at_indices(&aggregate_expressions, &indices)?; + let filter_expressions = get_at_indices(&filter_expressions, &indices)?; + // let accumulators = get_at_indices(&accumulators, &indices)?; + let accumulators = create_accumulators(&aggr_exprs)?; + Ok(AggregateGroup { + aggregate_expressions, + filter_expressions, + accumulators, + requirement, + }) + }) + .collect::>>()?; + // let aggregate_groups = vec![AggregateGroup { + // aggregate_expressions, + // filter_expressions, + // accumulators, + // }]; let inner = AggregateStreamInner { schema: Arc::clone(&agg.schema), mode: agg.mode, input, baseline_metrics, - aggregate_expressions, - filter_expressions, - accumulators, + aggregate_groups, reservation, finished: false, }; @@ -116,9 +145,7 @@ impl AggregateStream { let result = aggregate_batch( &this.mode, batch, - &mut this.accumulators, - &this.aggregate_expressions, - &this.filter_expressions, + &mut this.aggregate_groups, ); timer.done(); @@ -137,12 +164,15 @@ impl AggregateStream { None => { this.finished = true; let timer = this.baseline_metrics.elapsed_compute().timer(); - let result = finalize_aggregation(&this.accumulators, &this.mode) - .and_then(|columns| { - RecordBatch::try_new(this.schema.clone(), columns) - .map_err(Into::into) - }) - .record_output(&this.baseline_metrics); + let result = finalize_aggregation_groups( + &this.aggregate_groups, + &this.mode, + ) + .and_then(|columns| { + RecordBatch::try_new(this.schema.clone(), columns) + .map_err(Into::into) + }) + .record_output(&this.baseline_metrics); timer.done(); @@ -192,9 +222,9 @@ impl RecordBatchStream for AggregateStream { fn aggregate_batch( mode: &AggregateMode, batch: RecordBatch, - accumulators: &mut [AccumulatorItem], - expressions: &[Vec>], - filters: &[Option>], + aggregate_groups: &mut [AggregateGroup], + // expressions: &[Vec>], + // filters: &[Option>], ) -> Result { let mut allocated = 0usize; @@ -203,39 +233,52 @@ fn aggregate_batch( // 1.3 evaluate expressions // 1.4 update / merge accumulators with the expressions' values - // 1.1 - accumulators + aggregate_groups .iter_mut() - .zip(expressions) - .zip(filters) - .try_for_each(|((accum, expr), filter)| { - // 1.2 - let batch = match filter { - Some(filter) => Cow::Owned(batch_filter(&batch, filter)?), - None => Cow::Borrowed(&batch), + .try_for_each(|aggregate_group| { + let accumulators = &mut aggregate_group.accumulators; + let expressions = &mut aggregate_group.aggregate_expressions; + let filters = &mut aggregate_group.filter_expressions; + let requirement = &aggregate_group.requirement; + let batch = if requirement.is_empty() { + batch.clone() + } else { + sort_batch(&batch, requirement, None)? }; - // 1.3 - let values = &expr - .iter() - .map(|e| { - e.evaluate(&batch) - .and_then(|v| v.into_array(batch.num_rows())) + // 1.1 + accumulators + .iter_mut() + .zip(expressions) + .zip(filters) + .try_for_each(|((accum, expr), filter)| { + // 1.2 + let batch = match filter { + Some(filter) => Cow::Owned(batch_filter(&batch, filter)?), + None => Cow::Borrowed(&batch), + }; + // 1.3 + let values = &expr + .iter() + .map(|e| { + e.evaluate(&batch) + .and_then(|v| v.into_array(batch.num_rows())) + }) + .collect::>>()?; + + // 1.4 + let size_pre = accum.size(); + let res = match mode { + AggregateMode::Partial + | AggregateMode::Single + | AggregateMode::SinglePartitioned => accum.update_batch(values), + AggregateMode::Final | AggregateMode::FinalPartitioned => { + accum.merge_batch(values) + } + }; + let size_post = accum.size(); + allocated += size_post.saturating_sub(size_pre); + res }) - .collect::>>()?; - - // 1.4 - let size_pre = accum.size(); - let res = match mode { - AggregateMode::Partial - | AggregateMode::Single - | AggregateMode::SinglePartitioned => accum.update_batch(values), - AggregateMode::Final | AggregateMode::FinalPartitioned => { - accum.merge_batch(values) - } - }; - let size_post = accum.size(); - allocated += size_post.saturating_sub(size_pre); - res })?; Ok(allocated) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f96417fc323b..8fc369b76a61 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -24,8 +24,8 @@ use std::vec; use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ - evaluate_group_by, evaluate_many, evaluate_optional, group_schema, AggregateMode, - PhysicalGroupBy, + evaluate_group_by, evaluate_many, evaluate_optional, get_groups_indices, + group_schema, AggregateMode, PhysicalGroupBy, }; use crate::common::IPCWriter; use crate::metrics::{BaselineMetrics, RecordOutput}; @@ -36,6 +36,7 @@ use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; +use arrow::util::pretty::print_batches; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use arrow_schema::SortOptions; use datafusion_common::{DataFusionError, Result}; @@ -46,13 +47,17 @@ use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ - AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, + AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, LexOrdering, + PhysicalSortExpr, }; +use datafusion_common::utils::get_at_indices; use futures::ready; use futures::stream::{Stream, StreamExt}; use log::debug; +const PRINT_ON: bool = false; + #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) pub(crate) enum ExecutionState { @@ -208,30 +213,30 @@ pub(crate) struct GroupedHashAggregateStream { input: SendableRecordBatchStream, mode: AggregateMode, - /// Accumulators, one for each `AggregateExpr` in the query - /// - /// For example, if the query has aggregates, `SUM(x)`, - /// `COUNT(y)`, there will be two accumulators, each one - /// specialized for that particular aggregate and its input types - accumulators: Vec>, - - /// Arguments to pass to each accumulator. - /// - /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` - /// - /// The argument to each accumulator is itself a `Vec` because - /// some aggregates such as `CORR` can accept more than one - /// argument. - aggregate_arguments: Vec>>, - - /// Optional filter expression to evaluate, one for each for - /// accumulator. If present, only those rows for which the filter - /// evaluate to true should be included in the aggregate results. - /// - /// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`, - /// the filter expression is `x > 100`. - filter_expressions: Vec>>, - + aggregate_groups: Vec, + // /// Accumulators, one for each `AggregateExpr` in the query + // /// + // /// For example, if the query has aggregates, `SUM(x)`, + // /// `COUNT(y)`, there will be two accumulators, each one + // /// specialized for that particular aggregate and its input types + // accumulators: Vec>, + // + // /// Arguments to pass to each accumulator. + // /// + // /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` + // /// + // /// The argument to each accumulator is itself a `Vec` because + // /// some aggregates such as `CORR` can accept more than one + // /// argument. + // aggregate_arguments: Vec>>, + // + // /// Optional filter expression to evaluate, one for each for + // /// accumulator. If present, only those rows for which the filter + // /// evaluate to true should be included in the aggregate results. + // /// + // /// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`, + // /// the filter expression is `x > 100`. + // filter_expressions: Vec>>, /// GROUP BY expressions group_by: PhysicalGroupBy, @@ -295,6 +300,8 @@ impl GroupedHashAggregateStream { let aggregate_exprs = agg.aggr_expr.clone(); + let group_indices = get_groups_indices(&aggregate_exprs, agg.group_by()); + // arguments for each aggregate, one vec of expressions per // aggregate let aggregate_arguments = aggregates::aggregate_expressions( @@ -318,11 +325,42 @@ impl GroupedHashAggregateStream { } }; - // Instantiate the accumulators - let accumulators: Vec<_> = aggregate_exprs - .iter() - .map(create_group_accumulator) - .collect::>()?; + let mut aggregate_groups = group_indices + .into_iter() + .map(|(indices, mut requirement)| { + let aggregate_arguments = get_at_indices(&aggregate_arguments, &indices)?; + let merging_aggregate_arguments = + get_at_indices(&merging_aggregate_arguments, &indices)?; + let filter_expressions = get_at_indices(&filter_expressions, &indices)?; + let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; + // Instantiate the accumulators + let accumulators: Vec<_> = aggr_exprs + .iter() + .map(create_group_accumulator) + .collect::>()?; + // For final stages there is no requirement + if !agg.mode.is_first_stage() { + requirement.clear(); + } + + Ok(HashAggregateGroup { + accumulators, + aggregate_arguments, + merging_aggregate_arguments, + filter_expressions, + requirement, + }) + }) + .collect::>>()?; + if aggregate_groups.is_empty() { + aggregate_groups = vec![HashAggregateGroup { + accumulators: vec![], + aggregate_arguments: vec![], + merging_aggregate_arguments: vec![], + filter_expressions: vec![], + requirement: vec![], + }] + } let group_schema = group_schema(&agg_schema, agg_group_by.expr.len()); let spill_expr = group_schema @@ -366,9 +404,7 @@ impl GroupedHashAggregateStream { schema: agg_schema, input, mode: agg.mode, - accumulators, - aggregate_arguments, - filter_expressions, + aggregate_groups, group_by: agg_group_by, reservation, group_values, @@ -385,6 +421,34 @@ impl GroupedHashAggregateStream { } } +pub struct HashAggregateGroup { + /// Accumulators, one for each `AggregateExpr` in the query + /// + /// For example, if the query has aggregates, `SUM(x)`, + /// `COUNT(y)`, there will be two accumulators, each one + /// specialized for that particular aggregate and its input types + accumulators: Vec>, + + /// Arguments to pass to each accumulator. + /// + /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` + /// + /// The argument to each accumulator is itself a `Vec` because + /// some aggregates such as `CORR` can accept more than one + /// argument. + aggregate_arguments: Vec>>, + merging_aggregate_arguments: Vec>>, + + /// Optional filter expression to evaluate, one for each for + /// accumulator. If present, only those rows for which the filter + /// evaluate to true should be included in the aggregate results. + /// + /// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`, + /// the filter expression is `x > 100`. + filter_expressions: Vec>>, + requirement: LexOrdering, +} + /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. @@ -520,79 +584,99 @@ impl RecordBatchStream for GroupedHashAggregateStream { impl GroupedHashAggregateStream { /// Perform group-by aggregation for the given [`RecordBatch`]. fn group_aggregate_batch(&mut self, batch: RecordBatch) -> Result<()> { - // Evaluate the grouping expressions - let group_by_values = if self.spill_state.is_stream_merging { - evaluate_group_by(&self.spill_state.merging_group_by, &batch)? - } else { - evaluate_group_by(&self.group_by, &batch)? - }; - - // Evaluate the aggregation expressions. - let input_values = if self.spill_state.is_stream_merging { - evaluate_many(&self.spill_state.merging_aggregate_arguments, &batch)? - } else { - evaluate_many(&self.aggregate_arguments, &batch)? - }; - - // Evaluate the filter expressions, if any, against the inputs - let filter_values = if self.spill_state.is_stream_merging { - let filter_expressions = vec![None; self.accumulators.len()]; - evaluate_optional(&filter_expressions, &batch)? - } else { - evaluate_optional(&self.filter_expressions, &batch)? - }; - - for group_values in &group_by_values { - // calculate the group indices for each input row - let starting_num_groups = self.group_values.len(); - self.group_values - .intern(group_values, &mut self.current_group_indices)?; - let group_indices = &self.current_group_indices; - - // Update ordering information if necessary - let total_num_groups = self.group_values.len(); - if total_num_groups > starting_num_groups { - self.group_ordering.new_groups( - group_values, - group_indices, - total_num_groups, - )?; + for aggregate_group in &mut self.aggregate_groups { + if PRINT_ON { + println!( + "aggregate_group.requirement: {:?}", + aggregate_group.requirement + ); + println!("before sort"); + print_batches(&[batch.clone()])?; } + let batch = if aggregate_group.requirement.is_empty() { + batch.clone() + } else { + sort_batch(&batch, &aggregate_group.requirement, None)? + }; + if PRINT_ON { + print_batches(&[batch.clone()])?; + println!("after sort"); + } + // Evaluate the grouping expressions + let group_by_values = if self.spill_state.is_stream_merging { + evaluate_group_by(&self.spill_state.merging_group_by, &batch)? + } else { + evaluate_group_by(&self.group_by, &batch)? + }; + + // Evaluate the aggregation expressions. + let input_values = if self.spill_state.is_stream_merging { + evaluate_many(&aggregate_group.merging_aggregate_arguments, &batch)? + } else { + evaluate_many(&aggregate_group.aggregate_arguments, &batch)? + }; + + // Evaluate the filter expressions, if any, against the inputs + let filter_values = if self.spill_state.is_stream_merging { + let filter_expressions = vec![None; aggregate_group.accumulators.len()]; + evaluate_optional(&filter_expressions, &batch)? + } else { + evaluate_optional(&aggregate_group.filter_expressions, &batch)? + }; + + for group_values in &group_by_values { + // calculate the group indices for each input row + let starting_num_groups = self.group_values.len(); + self.group_values + .intern(group_values, &mut self.current_group_indices)?; + let group_indices = &self.current_group_indices; + + // Update ordering information if necessary + let total_num_groups = self.group_values.len(); + if total_num_groups > starting_num_groups { + self.group_ordering.new_groups( + group_values, + group_indices, + total_num_groups, + )?; + } - // Gather the inputs to call the actual accumulator - let t = self - .accumulators - .iter_mut() - .zip(input_values.iter()) - .zip(filter_values.iter()); - - for ((acc, values), opt_filter) in t { - let opt_filter = opt_filter.as_ref().map(|filter| filter.as_boolean()); - - // Call the appropriate method on each aggregator with - // the entire input row and the relevant group indexes - match self.mode { - AggregateMode::Partial - | AggregateMode::Single - | AggregateMode::SinglePartitioned - if !self.spill_state.is_stream_merging => - { - acc.update_batch( - values, - group_indices, - opt_filter, - total_num_groups, - )?; - } - _ => { - // if aggregation is over intermediate states, - // use merge - acc.merge_batch( - values, - group_indices, - opt_filter, - total_num_groups, - )?; + // Gather the inputs to call the actual accumulator + let t = aggregate_group + .accumulators + .iter_mut() + .zip(input_values.iter()) + .zip(filter_values.iter()); + + for ((acc, values), opt_filter) in t { + let opt_filter = + opt_filter.as_ref().map(|filter| filter.as_boolean()); + + // Call the appropriate method on each aggregator with + // the entire input row and the relevant group indexes + match self.mode { + AggregateMode::Partial + | AggregateMode::Single + | AggregateMode::SinglePartitioned + if !self.spill_state.is_stream_merging => + { + acc.update_batch( + values, + group_indices, + opt_filter, + total_num_groups, + )?; + } + _ => { + // if aggregation is over intermediate states, + // use merge + acc.merge_batch( + values, + group_indices, + opt_filter, + total_num_groups, + )?; + } } } } @@ -611,12 +695,19 @@ impl GroupedHashAggregateStream { } fn update_memory_reservation(&mut self) -> Result<()> { - let acc = self.accumulators.iter().map(|x| x.size()).sum::(); - self.reservation.try_resize( - acc + self.group_values.size() - + self.group_ordering.size() - + self.current_group_indices.allocated_size(), - ) + for aggregate_group in &self.aggregate_groups { + let acc = aggregate_group + .accumulators + .iter() + .map(|x| x.size()) + .sum::(); + self.reservation.try_resize( + acc + self.group_values.size() + + self.group_ordering.size() + + self.current_group_indices.allocated_size(), + )?; + } + Ok(()) } /// Create an output RecordBatch with the group keys and @@ -637,21 +728,35 @@ impl GroupedHashAggregateStream { } // Next output each aggregate value - for acc in self.accumulators.iter_mut() { - match self.mode { - AggregateMode::Partial => output.extend(acc.state(emit_to)?), - _ if spilling => { - // If spilling, output partial state because the spilled data will be - // merged and re-evaluated later. - output.extend(acc.state(emit_to)?) + let outputs = self + .aggregate_groups + .iter_mut() + .map(|aggregate_group| { + let mut aggregate_group_output = vec![]; + for acc in aggregate_group.accumulators.iter_mut() { + match self.mode { + AggregateMode::Partial => { + aggregate_group_output.extend(acc.state(emit_to)?) + } + _ if spilling => { + // If spilling, output partial state because the spilled data will be + // merged and re-evaluated later. + aggregate_group_output.extend(acc.state(emit_to)?) + } + AggregateMode::Final + | AggregateMode::FinalPartitioned + | AggregateMode::Single + | AggregateMode::SinglePartitioned => { + aggregate_group_output.push(acc.evaluate(emit_to)?) + } + } } - AggregateMode::Final - | AggregateMode::FinalPartitioned - | AggregateMode::Single - | AggregateMode::SinglePartitioned => output.push(acc.evaluate(emit_to)?), - } - } - + Ok(aggregate_group_output) + }) + .collect::>>()?; + // TODO: Consider proper indices during merging. + let aggregate_outputs = outputs.into_iter().flatten().collect::>(); + output.extend(aggregate_outputs); // emit reduces the memory usage. Ignore Err from update_memory_reservation. Even if it is // over the target memory size after emission, we can emit again rather than returning Err. let _ = self.update_memory_reservation(); diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index c97b9264f6d9..98e038744f6f 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,16 +2019,15 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) ---------------SortExec: expr=[col0@3 ASC NULLS LAST] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[3] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2212,10 +2211,10 @@ SELECT a, b, LAST_VALUE(c ORDER BY a DESC) as last_c FROM annotated_data_infinite2 GROUP BY a, b ---- -0 0 0 -0 1 25 -1 2 50 -1 3 75 +0 0 24 +0 1 49 +1 2 74 +1 3 99 # when LAST_VALUE, or FIRST_VALUE value do not contain ordering requirement # queries should still work, However, result depends on the scanning order and @@ -2296,8 +2295,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query T? @@ -2326,18 +2324,17 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM sales_global AS s GROUP BY s.country ---- FRA [200.0, 50.0] 250 -TUR [100.0, 75.0] 175 GRC [80.0, 30.0] 110 +TUR [100.0, 75.0] 175 query TT EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2359,7 +2356,7 @@ SortExec: expr=[country@0 ASC NULLS LAST] ----AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), ARRAY_AGG(s.amount), ARRAY_AGG(s.amount)] ------MemoryExec: partitions=1, partition_sizes=[1] -query T??? +query T??? rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, ARRAY_AGG(s.amount ORDER BY s.amount ASC) AS amounts2, ARRAY_AGG(s.amount ORDER BY s.sn ASC) AS amounts3 @@ -2392,10 +2389,10 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2428,10 +2425,10 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N physical_plan ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] --AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) -----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] +----SortExec: expr=[country@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] -query TI?R +query TI?R rowsort SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2467,7 +2464,7 @@ ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.coun ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2499,10 +2496,10 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] -query T?R +query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.country DESC, s.amount DESC) AS amounts, SUM(s.amount) AS sum1 FROM (SELECT * @@ -2530,11 +2527,10 @@ 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)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----MemoryExec: partitions=1, partition_sizes=[1] -query T?RR +query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, LAST_VALUE(amount ORDER BY amount DESC) AS fv2 @@ -2542,8 +2538,8 @@ SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, GROUP BY country ---- FRA [200.0, 50.0] 50 50 -TUR [100.0, 75.0] 75 75 GRC [80.0, 30.0] 30 30 +TUR [100.0, 75.0] 75 75 # test_reverse_aggregate_expr2 # Some of the Aggregators can be reversed, by this way we can still run aggregators without re-ordering @@ -2561,9 +2557,8 @@ 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)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2593,9 +2588,8 @@ 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)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), ARRAY_AGG(sales_global.amount)] +----MemoryExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2624,8 +2618,7 @@ Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_globa physical_plan ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@2 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2658,11 +2651,10 @@ 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)] -----SortExec: expr=[ts@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +--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 +query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, LAST_VALUE(amount ORDER BY ts DESC) as lv1, SUM(amount ORDER BY ts DESC) as sum1 @@ -2671,8 +2663,8 @@ SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, ORDER BY ts ASC) GROUP BY country ---- -GRC 80 30 110 FRA 200 50 250 +GRC 80 30 110 TUR 100 75 175 # If existing ordering doesn't satisfy requirement, we should do calculations @@ -2693,8 +2685,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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)] -----SortExec: expr=[ts@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TRRR SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2730,12 +2721,11 @@ physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] --ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] ----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] -------SortExec: expr=[sn@5 ASC NULLS LAST] ---------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] -----------CoalesceBatchesExec: target_batch_size=8192 -------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] +--------CoalesceBatchesExec: target_batch_size=8192 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2780,8 +2770,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------SortExec: expr=[ts@1 ASC NULLS LAST] -------------------MemoryExec: partitions=1, partition_sizes=[1] +----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2812,13 +2801,12 @@ physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] -----------------SortExec: expr=[ts@1 ASC NULLS LAST] -------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2852,9 +2840,8 @@ ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2876,12 +2863,11 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS ----TableScan: sales_global projection=[ts, amount] physical_plan ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] +--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2904,9 +2890,8 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts A --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -2928,9 +2913,8 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts D --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[ts@0 DESC] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -2952,9 +2936,8 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amou --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[amount@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -2982,9 +2965,8 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ---------------SortExec: expr=[amount@1 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3014,13 +2996,12 @@ physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------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)] ---------------SortExec: expr=[amount@1 DESC] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3645,10 +3626,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 @@ -4132,10 +4113,10 @@ Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULL ----TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 ------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST ---------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +--------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true @@ -4234,11 +4215,11 @@ Sort: multiple_ordered_table.a ASC NULLS LAST, multiple_ordered_table.b ASC NULL ----TableScan: multiple_ordered_table projection=[a, b, c, d] physical_plan SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ---AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +--AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=20 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 -----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(multiple_ordered_table.d), FIRST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted +----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true From 5c05f5033e84a3c9003671d738051a3de936e708 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 13:50:13 +0300 Subject: [PATCH 15/49] Implement grouping algorithm --- .../physical-plan/src/aggregates/mod.rs | 121 +++++++++++++++--- .../src/aggregates/no_grouping.rs | 14 +- .../physical-plan/src/aggregates/row_hash.rs | 11 +- 3 files changed, 121 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bca74b9b05b9..fb6c788fff49 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -50,6 +50,7 @@ use datafusion_physical_expr::{ LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use futures::StreamExt; +use hashbrown::HashSet; use itertools::{izip, Itertools}; @@ -998,26 +999,71 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } +fn get_req( + aggr_expr: &Arc, + group_by: &PhysicalGroupBy, +) -> LexOrdering { + let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + if !is_order_sensitive(aggr_expr) + || group_by_contains_all_requirements(&group_by, &req) + { + // No requirement. + req.clear(); + } + req +} + fn get_groups_indices( - aggr_exprs: &[Arc], + aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, + eq_properties: &EquivalenceProperties, ) -> Vec<(Vec, LexOrdering)> { let mut initial_groups = vec![]; for idx in 0..aggr_exprs.len() { let aggr_expr = &aggr_exprs[idx]; - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); - if !is_order_sensitive(aggr_expr) - || group_by_contains_all_requirements(&group_by, &req) - { - // No requirement. - req.clear(); - } - // let req = aggr_expr[idx].order_bys().unwrap_or_default().to_vec(); + let req = get_req(&aggr_expr, group_by); initial_groups.push((vec![idx], req)); } - // TODO: Add merge groups logic. - initial_groups + let mut used_indices: HashSet = HashSet::new(); + let mut groups = vec![]; + while used_indices.len() != aggr_exprs.len() { + let mut group: Option<(Vec, LexOrdering)> = None; + for idx in 0..aggr_exprs.len() { + let aggr_expr = &mut aggr_exprs[idx]; + // Group is empty and index is not already in another group. + if used_indices.contains(&idx) { + // Skip this group, it is already inserted. + continue; + } + let aggr_req = get_req(&aggr_expr, group_by); + if let Some((group_indices, req)) = &mut group { + if let Some(finer) = eq_properties.get_finer_ordering(req, &aggr_req) { + *req = finer; + group_indices.push(idx); + } else if let Some(reverse) = aggr_expr.reverse_expr() { + let reverse_req = get_req(&reverse, group_by); + if let Some(finer) = + eq_properties.get_finer_ordering(req, &reverse_req) + { + *aggr_expr = reverse; + *req = finer; + group_indices.push(idx); + } + } + } else { + group = Some((vec![idx], aggr_req)); + } + } + if let Some((group_indices, req)) = group { + used_indices.extend(group_indices.iter()); + groups.push((group_indices, req)); + } else { + unreachable!(); + } + } + + groups } /// returns physical expressions for arguments to evaluate against a batch @@ -1288,6 +1334,7 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use futures::{FutureExt, Stream}; + use itertools::GroupBy; // Generate a schema which consists of 5 columns (a, b, c, d, e) fn create_test_schema() -> Result { @@ -2259,7 +2306,7 @@ mod tests { nulls_first: false, }; // This is the reverse requirement of options1 - let options_dess = SortOptions { + let option_desc = SortOptions { descending: true, nulls_first: true, }; @@ -2273,11 +2320,45 @@ mod tests { // Ordering requirements vec![vec![(col_a, option_asc)]], // expected - vec![vec![0]], + vec![(vec![0], vec![(col_a, option_asc)])], + ), + // ------- TEST CASE 2 ----------- + ( + // Ordering requirements + vec![vec![(col_a, option_asc)], vec![(col_a, option_asc)]], + // expected + vec![(vec![0, 1], vec![(col_a, option_asc)])], + ), + // ------- TEST CASE 3 ----------- + ( + // Ordering requirements + vec![ + vec![(col_a, option_asc), (col_b, option_asc)], + vec![(col_a, option_asc)], + ], + // expected + vec![(vec![0, 1], vec![(col_a, option_asc), (col_b, option_asc)])], + ), + // ------- TEST CASE 4 ----------- + ( + // Ordering requirements + vec![vec![(col_a, option_asc)], vec![(col_a, option_desc)]], + // expected + vec![(vec![0, 1], vec![(col_a, option_asc)])], + ), + // ------- TEST CASE 5 ----------- + ( + // Ordering requirements + vec![vec![(col_a, option_asc)], vec![(col_c, option_asc)]], + // expected + vec![ + (vec![0], vec![(col_a, option_asc)]), + (vec![1], vec![(col_c, option_asc)]), + ], ), ]; for (ordering_reqs, expected) in test_cases { - let aggr_exprs = ordering_reqs + let mut aggr_exprs = ordering_reqs .into_iter() .map(|req| { let req = convert_to_sort_exprs(&req); @@ -2292,7 +2373,17 @@ mod tests { }) .collect::>(); - let res = get_groups_indices(&aggr_exprs); + let group_by = PhysicalGroupBy::new_single(vec![]); + + // Empty equivalence Properties + let eq_properties = EquivalenceProperties::new(test_schema.clone()); + + let res = get_groups_indices(&mut aggr_exprs, &group_by, &eq_properties); + + let expected = expected + .into_iter() + .map(|(indices, req)| (indices, convert_to_sort_exprs(&req))) + .collect::>(); assert_eq!(res, expected); } Ok(()) diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 8a0b3b9a4cba..c30ee5c7c261 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -23,7 +23,7 @@ use crate::aggregates::{ AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; -use crate::{RecordBatchStream, SendableRecordBatchStream}; +use crate::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; @@ -86,8 +86,12 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - let group_indices = get_groups_indices(&agg.aggr_expr, agg.group_by()); - let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; + let mut aggregate_exprs = agg.aggr_expr.to_vec(); + let eq_properties = agg.equivalence_properties(); + let group_indices = + get_groups_indices(&mut aggregate_exprs, agg.group_by(), &eq_properties); + let aggregate_expressions = + aggregate_expressions(&aggregate_exprs, &agg.mode, 0)?; let filter_expressions = match agg.mode { AggregateMode::Partial | AggregateMode::Single @@ -96,14 +100,14 @@ impl AggregateStream { vec![None; agg.aggr_expr.len()] } }; - let accumulators = create_accumulators(&agg.aggr_expr)?; + let accumulators = create_accumulators(&aggregate_exprs)?; let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); let aggregate_groups = group_indices .into_iter() .map(|(indices, requirement)| { - let aggr_exprs = get_at_indices(&agg.aggr_expr, &indices)?; + let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; let aggregate_expressions = get_at_indices(&aggregate_expressions, &indices)?; let filter_expressions = get_at_indices(&filter_expressions, &indices)?; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 8fc369b76a61..84685346800e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -298,20 +298,21 @@ impl GroupedHashAggregateStream { let timer = baseline_metrics.elapsed_compute().timer(); - let aggregate_exprs = agg.aggr_expr.clone(); - - let group_indices = get_groups_indices(&aggregate_exprs, agg.group_by()); + let mut aggregate_exprs = agg.aggr_expr.clone(); + let eq_properties = agg.equivalence_properties(); + let group_indices = + get_groups_indices(&mut aggregate_exprs, agg.group_by(), &eq_properties); // arguments for each aggregate, one vec of expressions per // aggregate let aggregate_arguments = aggregates::aggregate_expressions( - &agg.aggr_expr, + &aggregate_exprs, &agg.mode, agg_group_by.expr.len(), )?; // arguments for aggregating spilled data is the same as the one for final aggregation let merging_aggregate_arguments = aggregates::aggregate_expressions( - &agg.aggr_expr, + &aggregate_exprs, &AggregateMode::Final, agg_group_by.expr.len(), )?; From d7ead9369b5dc8b293113979b7d45eb656cb3e3c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 13:55:05 +0300 Subject: [PATCH 16/49] Minor changes --- datafusion/physical-plan/src/aggregates/mod.rs | 15 +++------------ 1 file changed, 3 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index fb6c788fff49..b383b1d4b8ef 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1082,18 +1082,9 @@ fn aggregate_expressions( .iter() .map(|agg| { let mut result = agg.expressions().clone(); - // In partial mode, append ordering requirements to expressions' results. - // Ordering requirements are used by subsequent executors to satisfy the required - // ordering for `AggregateMode::FinalPartitioned`/`AggregateMode::Final` modes. - // if matches!(mode, AggregateMode::Partial) { - // if let Some(ordering_req) = agg.order_bys() { - // let ordering_exprs = ordering_req - // .iter() - // .map(|item| item.expr.clone()) - // .collect::>(); - // result.extend(ordering_exprs); - // } - // } + // Append ordering requirements to expressions' results. + // This way order sensitive aggregators can satisfy requirement + // themselves. if let Some(ordering_req) = agg.order_bys() { let ordering_exprs = ordering_req .iter() From 019b9ba6cee1001232ea14490c40828811845eb4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 14:06:52 +0300 Subject: [PATCH 17/49] Add reverse support --- .../src/physical_optimizer/enforce_sorting.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 34 +++++++----- .../src/aggregates/no_grouping.rs | 38 +++++++------ .../physical-plan/src/aggregates/row_hash.rs | 55 +++++++++++-------- 4 files changed, 75 insertions(+), 56 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index be6b6853c73f..2265724c6f80 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2659,7 +2659,7 @@ LOCATION '../core/tests/data/window_2.csv';"; let expected = vec![ "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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as lv2]", - " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)]", + " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", " MemoryExec: partitions=1, partition_sizes=[1]", ]; // Get string representation of the plan @@ -2709,7 +2709,7 @@ LOCATION '../core/tests/data/window_2.csv';"; "ProjectionExec: expr=[fv1@0 as fv1, lv1@1 as lv1, fv2@2 as fv2, lv2@3 as lv2]", " SortExec: expr=[country@4 ASC NULLS LAST]", " ProjectionExec: expr=[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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@4 as lv2, country@0 as country]", - " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)]", + " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", " MemoryExec: partitions=1, partition_sizes=[1]", ]; // Get string representation of the plan diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b383b1d4b8ef..ffd7628ff073 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -271,16 +271,6 @@ impl From for SendableRecordBatchStream { } } -// pub struct AggregateGroup{ -// /// Aggregate expressions -// aggr_expr: Vec>, -// /// FILTER (WHERE clause) expression for each aggregate expression -// filter_expr: Vec>>, -// /// (ORDER BY clause) expression for each aggregate expression -// /// TODO: Make below variable LexOrdering -// order_by_expr: Vec>, -// } - pub struct AggregateGroup { aggregate_expressions: Vec>>, filter_expressions: Vec>>, @@ -288,6 +278,14 @@ pub struct AggregateGroup { requirement: LexOrdering, } +#[derive(Debug)] +pub struct AggregateExprGroup { + /// Aggregate expressions indices + indices: Vec, + /// Requirement + requirement: LexOrdering, +} + /// Hash aggregate execution plan #[derive(Debug)] pub struct AggregateExec { @@ -301,6 +299,7 @@ pub struct AggregateExec { filter_expr: Vec>>, /// (ORDER BY clause) expression for each aggregate expression order_by_expr: Vec>, + aggregate_groups: Vec, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause limit: Option, /// Input plan, could be a partial aggregate or the input to the aggregate @@ -497,6 +496,9 @@ impl AggregateExec { group_by.contains_null(), mode, )?; + let eq_properties = input.equivalence_properties(); + let aggregate_groups = + get_groups_indices(&mut aggr_expr, &group_by, &eq_properties); let schema = Arc::new(schema); // Reset ordering requirement to `None` if aggregator is not order-sensitive @@ -572,6 +574,7 @@ impl AggregateExec { aggr_expr, filter_expr, order_by_expr, + aggregate_groups, input, schema, input_schema, @@ -1017,7 +1020,7 @@ fn get_groups_indices( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, -) -> Vec<(Vec, LexOrdering)> { +) -> Vec { let mut initial_groups = vec![]; for idx in 0..aggr_exprs.len() { let aggr_expr = &aggr_exprs[idx]; @@ -1055,9 +1058,12 @@ fn get_groups_indices( group = Some((vec![idx], aggr_req)); } } - if let Some((group_indices, req)) = group { - used_indices.extend(group_indices.iter()); - groups.push((group_indices, req)); + if let Some((indices, requirement)) = group { + used_indices.extend(indices.iter()); + groups.push(AggregateExprGroup { + indices, + requirement, + }); } else { unreachable!(); } diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index c30ee5c7c261..8b8326935d3b 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -19,8 +19,8 @@ use crate::aggregates::{ aggregate_expressions, create_accumulators, finalize_aggregation, - finalize_aggregation_groups, get_groups_indices, AccumulatorItem, AggregateGroup, - AggregateMode, + finalize_aggregation_groups, get_groups_indices, AccumulatorItem, AggregateExprGroup, + AggregateGroup, AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; @@ -106,20 +106,26 @@ impl AggregateStream { .register(context.memory_pool()); let aggregate_groups = group_indices .into_iter() - .map(|(indices, requirement)| { - let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; - let aggregate_expressions = - get_at_indices(&aggregate_expressions, &indices)?; - let filter_expressions = get_at_indices(&filter_expressions, &indices)?; - // let accumulators = get_at_indices(&accumulators, &indices)?; - let accumulators = create_accumulators(&aggr_exprs)?; - Ok(AggregateGroup { - aggregate_expressions, - filter_expressions, - accumulators, - requirement, - }) - }) + .map( + |AggregateExprGroup { + indices, + requirement, + }| { + let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; + let aggregate_expressions = + get_at_indices(&aggregate_expressions, &indices)?; + let filter_expressions = + get_at_indices(&filter_expressions, &indices)?; + // let accumulators = get_at_indices(&accumulators, &indices)?; + let accumulators = create_accumulators(&aggr_exprs)?; + Ok(AggregateGroup { + aggregate_expressions, + filter_expressions, + accumulators, + requirement, + }) + }, + ) .collect::>>()?; // let aggregate_groups = vec![AggregateGroup { // aggregate_expressions, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 84685346800e..f3521c95b8ca 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -25,7 +25,7 @@ use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ evaluate_group_by, evaluate_many, evaluate_optional, get_groups_indices, - group_schema, AggregateMode, PhysicalGroupBy, + group_schema, AggregateExprGroup, AggregateMode, PhysicalGroupBy, }; use crate::common::IPCWriter; use crate::metrics::{BaselineMetrics, RecordOutput}; @@ -328,30 +328,37 @@ impl GroupedHashAggregateStream { let mut aggregate_groups = group_indices .into_iter() - .map(|(indices, mut requirement)| { - let aggregate_arguments = get_at_indices(&aggregate_arguments, &indices)?; - let merging_aggregate_arguments = - get_at_indices(&merging_aggregate_arguments, &indices)?; - let filter_expressions = get_at_indices(&filter_expressions, &indices)?; - let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; - // Instantiate the accumulators - let accumulators: Vec<_> = aggr_exprs - .iter() - .map(create_group_accumulator) - .collect::>()?; - // For final stages there is no requirement - if !agg.mode.is_first_stage() { - requirement.clear(); - } + .map( + |AggregateExprGroup { + indices, + mut requirement, + }| { + let aggregate_arguments = + get_at_indices(&aggregate_arguments, &indices)?; + let merging_aggregate_arguments = + get_at_indices(&merging_aggregate_arguments, &indices)?; + let filter_expressions = + get_at_indices(&filter_expressions, &indices)?; + let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; + // Instantiate the accumulators + let accumulators: Vec<_> = aggr_exprs + .iter() + .map(create_group_accumulator) + .collect::>()?; + // For final stages there is no requirement + if !agg.mode.is_first_stage() { + requirement.clear(); + } - Ok(HashAggregateGroup { - accumulators, - aggregate_arguments, - merging_aggregate_arguments, - filter_expressions, - requirement, - }) - }) + Ok(HashAggregateGroup { + accumulators, + aggregate_arguments, + merging_aggregate_arguments, + filter_expressions, + requirement, + }) + }, + ) .collect::>>()?; if aggregate_groups.is_empty() { aggregate_groups = vec![HashAggregateGroup { From e4b9ebc4e054b41a041de958d33d8d0db896cb87 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 14:23:26 +0300 Subject: [PATCH 18/49] Minor changes --- .../physical-plan/src/aggregates/mod.rs | 27 ++++++++++----- .../src/aggregates/no_grouping.rs | 32 +++++++---------- .../physical-plan/src/aggregates/row_hash.rs | 34 +++++++------------ .../sqllogictest/test_files/groupby.slt | 18 +++++----- 4 files changed, 53 insertions(+), 58 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ffd7628ff073..bb26a18ccf9f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -278,7 +278,7 @@ pub struct AggregateGroup { requirement: LexOrdering, } -#[derive(Debug)] +#[derive(Clone, Debug, PartialEq)] pub struct AggregateExprGroup { /// Aggregate expressions indices indices: Vec, @@ -498,7 +498,7 @@ impl AggregateExec { )?; let eq_properties = input.equivalence_properties(); let aggregate_groups = - get_groups_indices(&mut aggr_expr, &group_by, &eq_properties); + get_groups_indices(&mut aggr_expr, &group_by, &eq_properties, &mode); let schema = Arc::new(schema); // Reset ordering requirement to `None` if aggregator is not order-sensitive @@ -1005,12 +1005,14 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { fn get_req( aggr_expr: &Arc, group_by: &PhysicalGroupBy, + agg_mode: &AggregateMode, ) -> LexOrdering { let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); if !is_order_sensitive(aggr_expr) || group_by_contains_all_requirements(&group_by, &req) + // For final stages there is no requirement + || !agg_mode.is_first_stage() { - // No requirement. req.clear(); } req @@ -1020,11 +1022,12 @@ fn get_groups_indices( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, + agg_mode: &AggregateMode, ) -> Vec { let mut initial_groups = vec![]; for idx in 0..aggr_exprs.len() { let aggr_expr = &aggr_exprs[idx]; - let req = get_req(&aggr_expr, group_by); + let req = get_req(&aggr_expr, group_by, agg_mode); initial_groups.push((vec![idx], req)); } @@ -1039,13 +1042,13 @@ fn get_groups_indices( // Skip this group, it is already inserted. continue; } - let aggr_req = get_req(&aggr_expr, group_by); + let aggr_req = get_req(&aggr_expr, group_by, agg_mode); if let Some((group_indices, req)) = &mut group { if let Some(finer) = eq_properties.get_finer_ordering(req, &aggr_req) { *req = finer; group_indices.push(idx); } else if let Some(reverse) = aggr_expr.reverse_expr() { - let reverse_req = get_req(&reverse, group_by); + let reverse_req = get_req(&reverse, group_by, agg_mode); if let Some(finer) = eq_properties.get_finer_ordering(req, &reverse_req) { @@ -2375,11 +2378,19 @@ mod tests { // Empty equivalence Properties let eq_properties = EquivalenceProperties::new(test_schema.clone()); - let res = get_groups_indices(&mut aggr_exprs, &group_by, &eq_properties); + let res = get_groups_indices( + &mut aggr_exprs, + &group_by, + &eq_properties, + &AggregateMode::Partial, + ); let expected = expected .into_iter() - .map(|(indices, req)| (indices, convert_to_sort_exprs(&req))) + .map(|(indices, req)| AggregateExprGroup { + indices, + requirement: convert_to_sort_exprs(&req), + }) .collect::>(); assert_eq!(res, expected); } diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 8b8326935d3b..4301a1b559ad 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -19,8 +19,8 @@ use crate::aggregates::{ aggregate_expressions, create_accumulators, finalize_aggregation, - finalize_aggregation_groups, get_groups_indices, AccumulatorItem, AggregateExprGroup, - AggregateGroup, AggregateMode, + finalize_aggregation_groups, AccumulatorItem, AggregateExprGroup, AggregateGroup, + AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; @@ -86,12 +86,7 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; - let mut aggregate_exprs = agg.aggr_expr.to_vec(); - let eq_properties = agg.equivalence_properties(); - let group_indices = - get_groups_indices(&mut aggregate_exprs, agg.group_by(), &eq_properties); - let aggregate_expressions = - aggregate_expressions(&aggregate_exprs, &agg.mode, 0)?; + let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { AggregateMode::Partial | AggregateMode::Single @@ -100,38 +95,35 @@ impl AggregateStream { vec![None; agg.aggr_expr.len()] } }; - let accumulators = create_accumulators(&aggregate_exprs)?; + let accumulators = create_accumulators(&agg.aggr_expr)?; let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); - let aggregate_groups = group_indices - .into_iter() + let aggregate_groups = agg + .aggregate_groups + .iter() .map( |AggregateExprGroup { indices, requirement, }| { - let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; + let aggr_exprs = get_at_indices(&agg.aggr_expr, indices)?; let aggregate_expressions = - get_at_indices(&aggregate_expressions, &indices)?; + get_at_indices(&aggregate_expressions, indices)?; let filter_expressions = - get_at_indices(&filter_expressions, &indices)?; + get_at_indices(&filter_expressions, indices)?; // let accumulators = get_at_indices(&accumulators, &indices)?; let accumulators = create_accumulators(&aggr_exprs)?; Ok(AggregateGroup { aggregate_expressions, filter_expressions, accumulators, - requirement, + requirement: requirement.to_vec(), }) }, ) .collect::>>()?; - // let aggregate_groups = vec![AggregateGroup { - // aggregate_expressions, - // filter_expressions, - // accumulators, - // }]; + let inner = AggregateStreamInner { schema: Arc::clone(&agg.schema), mode: agg.mode, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f3521c95b8ca..2272ecb7ad8c 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -24,8 +24,8 @@ use std::vec; use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ - evaluate_group_by, evaluate_many, evaluate_optional, get_groups_indices, - group_schema, AggregateExprGroup, AggregateMode, PhysicalGroupBy, + evaluate_group_by, evaluate_many, evaluate_optional, group_schema, + AggregateExprGroup, AggregateMode, PhysicalGroupBy, }; use crate::common::IPCWriter; use crate::metrics::{BaselineMetrics, RecordOutput}; @@ -298,21 +298,16 @@ impl GroupedHashAggregateStream { let timer = baseline_metrics.elapsed_compute().timer(); - let mut aggregate_exprs = agg.aggr_expr.clone(); - let eq_properties = agg.equivalence_properties(); - - let group_indices = - get_groups_indices(&mut aggregate_exprs, agg.group_by(), &eq_properties); // arguments for each aggregate, one vec of expressions per // aggregate let aggregate_arguments = aggregates::aggregate_expressions( - &aggregate_exprs, + &agg.aggr_expr, &agg.mode, agg_group_by.expr.len(), )?; // arguments for aggregating spilled data is the same as the one for final aggregation let merging_aggregate_arguments = aggregates::aggregate_expressions( - &aggregate_exprs, + &agg.aggr_expr, &AggregateMode::Final, agg_group_by.expr.len(), )?; @@ -326,36 +321,33 @@ impl GroupedHashAggregateStream { } }; - let mut aggregate_groups = group_indices - .into_iter() + let mut aggregate_groups = agg + .aggregate_groups + .iter() .map( |AggregateExprGroup { indices, - mut requirement, + requirement, }| { let aggregate_arguments = - get_at_indices(&aggregate_arguments, &indices)?; + get_at_indices(&aggregate_arguments, indices)?; let merging_aggregate_arguments = - get_at_indices(&merging_aggregate_arguments, &indices)?; + get_at_indices(&merging_aggregate_arguments, indices)?; let filter_expressions = - get_at_indices(&filter_expressions, &indices)?; - let aggr_exprs = get_at_indices(&aggregate_exprs, &indices)?; + get_at_indices(&filter_expressions, indices)?; + let aggr_exprs = get_at_indices(&agg.aggr_expr, indices)?; // Instantiate the accumulators let accumulators: Vec<_> = aggr_exprs .iter() .map(create_group_accumulator) .collect::>()?; - // For final stages there is no requirement - if !agg.mode.is_first_stage() { - requirement.clear(); - } Ok(HashAggregateGroup { accumulators, aggregate_arguments, merging_aggregate_arguments, filter_expressions, - requirement, + requirement: requirement.to_vec(), }) }, ) diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 98e038744f6f..8ecefccfcded 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2527,7 +2527,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)] ----MemoryExec: partitions=1, partition_sizes=[1] query T?RR rowsort @@ -2557,7 +2557,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)] ----MemoryExec: partitions=1, partition_sizes=[1] query T?RR @@ -2588,7 +2588,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)] ----MemoryExec: partitions=1, partition_sizes=[1] query TRR? @@ -2801,11 +2801,11 @@ physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@1 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR @@ -2863,9 +2863,9 @@ Projection: FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS ----TableScan: sales_global projection=[ts, amount] physical_plan ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]@0 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as fv2] ---AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +--AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----CoalescePartitionsExec -------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] --------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ----------MemoryExec: partitions=1, partition_sizes=[1] @@ -2996,10 +2996,10 @@ physical_plan SortPreservingMergeExec: [country@0 ASC NULLS LAST] --SortExec: expr=[country@0 ASC NULLS LAST] ----ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@1 as amounts, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as fv1, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]@3 as fv2] -------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] +------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_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)] --------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ----------------MemoryExec: partitions=1, partition_sizes=[1] From e2ee90a0a3bd93bd451b1eb572fd45138d49ec0d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 14:24:22 +0300 Subject: [PATCH 19/49] Minor changes --- .../physical-expr/src/aggregate/first_last.rs | 47 ------------------- 1 file changed, 47 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index c79bcfc1e376..15f2d580378e 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -551,53 +551,6 @@ fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { .collect::>() } -// /// Gets either first, or last value index inside values columns according to ordering requirements -// fn get_value_idx( -// values: &[ArrayRef], -// ordering_req: &[PhysicalSortExpr], -// is_set: bool, -// ) -> Result> { -// let value = &values[0]; -// let ordering_values = &values[1..]; -// if value.is_empty() { -// // For empty batches there is nothing to update -// return Ok(None); -// } -// -// Ok(Some(if ordering_req.is_empty() && !is_set && FIRST { -// 0 -// } else if ordering_req.is_empty() && is_set && FIRST { -// // No need to overwrite existing value, when no ordering is specified -// // Just use first value encountered -// return Ok(None); -// } else if ordering_req.is_empty() && !FIRST { -// value.len() - 1 -// } else { -// // Calculate real first, or last value according to requirement. -// let sort_options = if FIRST { -// get_sort_options(ordering_req) -// } else { -// // last -// // Reverse requirement options (e.g last, is the first entry in the reverse order) -// ordering_req -// .iter() -// .map(|sort_expr| !sort_expr.options) -// .collect() -// }; -// -// let sort_columns = ordering_values -// .iter() -// .zip(sort_options) -// .map(|(col, options)| SortColumn { -// values: col.clone(), -// options: Some(options), -// }) -// .collect::>(); -// let indices = lexsort_to_indices(&sort_columns, Some(1))?; -// indices.value(0) as usize -// })) -// } - /// Gets either first, or last value index inside values columns according to ordering requirements fn get_value_idx( values: &[ArrayRef], From e0f2889b1f14b327596fefc152a884b7a2b7a085 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 15:10:21 +0300 Subject: [PATCH 20/49] All cli tests pass --- .../src/physical_optimizer/enforce_sorting.rs | 15 +- .../physical-plan/src/aggregates/mod.rs | 97 +++++++----- .../sqllogictest/test_files/groupby.slt | 141 +++++++++++------- 3 files changed, 150 insertions(+), 103 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 2265724c6f80..d9212434b927 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2660,7 +2660,8 @@ LOCATION '../core/tests/data/window_2.csv';"; let expected = vec![ "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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as lv2]", " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " SortExec: expr=[ts@0 ASC NULLS LAST]", + " MemoryExec: partitions=1, partition_sizes=[1]", ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); @@ -2710,7 +2711,8 @@ LOCATION '../core/tests/data/window_2.csv';"; " SortExec: expr=[country@4 ASC NULLS LAST]", " ProjectionExec: expr=[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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@4 as lv2, country@0 as country]", " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", - " MemoryExec: partitions=1, partition_sizes=[1]", + " SortExec: expr=[ts@1 ASC NULLS LAST]", + " MemoryExec: partitions=1, partition_sizes=[1]", ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); @@ -2845,7 +2847,8 @@ GROUP BY d"; let expected = vec![ "ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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=Single, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.c), LAST_VALUE(multiple_ordered_table.c)]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", + " SortExec: expr=[a@0 DESC,c@1 DESC]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); @@ -2899,9 +2902,9 @@ ORDER BY d"; " 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.c), LAST_VALUE(multiple_ordered_table.c)]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", - ]; + " SortExec: expr=[a@0 DESC,c@1 DESC]", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); assert_eq!( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bb26a18ccf9f..311bc58872a3 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -497,46 +497,56 @@ impl AggregateExec { mode, )?; let eq_properties = input.equivalence_properties(); - let aggregate_groups = + let mut aggregate_groups = get_groups_indices(&mut aggr_expr, &group_by, &eq_properties, &mode); let schema = Arc::new(schema); - // Reset ordering requirement to `None` if aggregator is not order-sensitive - order_by_expr = aggr_expr - .iter() - .zip(order_by_expr) - .map(|(aggr_expr, fn_reqs)| { - // If - // - aggregation function is order-sensitive and - // - aggregation is performing a "first stage" calculation, and - // - at least one of the aggregate function requirement is not inside group by expression - // keep the ordering requirement as is; otherwise ignore the ordering requirement. - // 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. - fn_reqs.filter(|req| { - is_order_sensitive(aggr_expr) - && mode.is_first_stage() - && !group_by_contains_all_requirements(&group_by, req) - }) - }) - .collect::>(); - let requirement = get_finest_requirement( - &mut aggr_expr, - &mut order_by_expr, - &input.equivalence_properties(), - )?; - let mut ordering_req = requirement.unwrap_or(vec![]); - let partition_search_mode = get_aggregate_search_mode( - &group_by, - &input, - &mut aggr_expr, - &mut order_by_expr, - &mut ordering_req, - ); + // // Reset ordering requirement to `None` if aggregator is not order-sensitive + // order_by_expr = aggr_expr + // .iter() + // .zip(order_by_expr) + // .map(|(aggr_expr, fn_reqs)| { + // // If + // // - aggregation function is order-sensitive and + // // - aggregation is performing a "first stage" calculation, and + // // - at least one of the aggregate function requirement is not inside group by expression + // // keep the ordering requirement as is; otherwise ignore the ordering requirement. + // // 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. + // fn_reqs.filter(|req| { + // is_order_sensitive(aggr_expr) + // && mode.is_first_stage() + // && !group_by_contains_all_requirements(&group_by, req) + // }) + // }) + // .collect::>(); + // let requirement = get_finest_requirement( + // &mut aggr_expr, + // &mut order_by_expr, + // &input.equivalence_properties(), + // )?; + // let mut ordering_req = requirement.unwrap_or(vec![]); + // let partition_search_mode = get_aggregate_search_mode( + // &group_by, + // &input, + // &mut aggr_expr, + // &mut order_by_expr, + // &mut ordering_req, + // ); + + let mut ordering_req = vec![]; + if let Some(max_group) = aggregate_groups + .iter_mut() + .max_by(|lhs, rhs| lhs.indices.len().cmp(&rhs.indices.len())) + { + ordering_req = max_group.requirement.to_vec(); + // No longer require this, this requirement will be satisfied by outside mechanism. + max_group.requirement.clear(); + } // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); @@ -545,8 +555,8 @@ impl AggregateExec { // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); let mut new_requirement = indices - .into_iter() - .map(|idx| PhysicalSortRequirement { + .iter() + .map(|&idx| PhysicalSortRequirement { expr: groupby_exprs[idx].clone(), options: None, }) @@ -556,6 +566,15 @@ impl AggregateExec { new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); + let partition_search_mode = + if indices.len() == groupby_exprs.len() && !groupby_exprs.is_empty() { + PartitionSearchMode::Sorted + } else if indices.len() > 0 { + PartitionSearchMode::PartiallySorted(indices) + } else { + PartitionSearchMode::Linear + }; + // construct a map from the input expression to the output expression of the Aggregation group by let projection_mapping = ProjectionMapping::try_new(&group_by.expr, &input.schema())?; diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 8ecefccfcded..aa1e0dcbafe0 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,15 +2019,16 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[3] -------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -----------------------MemoryExec: partitions=1, partition_sizes=[3] +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) +--------------SortExec: expr=[col0@3 ASC NULLS LAST] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[3] +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +------------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2295,7 +2296,8 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] -----MemoryExec: partitions=1, partition_sizes=[1] +----SortExec: expr=[amount@1 ASC NULLS LAST] +------MemoryExec: partitions=1, partition_sizes=[1] query T? @@ -2324,7 +2326,8 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] -----MemoryExec: partitions=1, partition_sizes=[1] +----SortExec: expr=[amount@1 DESC] +------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2354,7 +2357,8 @@ physical_plan SortExec: expr=[country@0 ASC NULLS LAST] --ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, ARRAY_AGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]@2 as amounts2, ARRAY_AGG(s.amount) ORDER BY [s.sn ASC NULLS LAST]@3 as amounts3] ----AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), ARRAY_AGG(s.amount), ARRAY_AGG(s.amount)] -------MemoryExec: partitions=1, partition_sizes=[1] +------SortExec: expr=[sn@1 ASC NULLS LAST] +--------MemoryExec: partitions=1, partition_sizes=[1] query T??? rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2389,7 +2393,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST] +----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2425,7 +2429,7 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N physical_plan ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] --AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) -----SortExec: expr=[country@1 ASC NULLS LAST] +----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] query TI?R rowsort @@ -2496,7 +2500,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST] +----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2528,7 +2532,8 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales 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)] -----MemoryExec: partitions=1, partition_sizes=[1] +----SortExec: expr=[amount@1 DESC] +------MemoryExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2558,7 +2563,8 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales 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)] -----MemoryExec: partitions=1, partition_sizes=[1] +----SortExec: expr=[amount@1 ASC NULLS LAST] +------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2589,7 +2595,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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)] -----MemoryExec: partitions=1, partition_sizes=[1] +----SortExec: expr=[amount@1 ASC NULLS LAST] +------MemoryExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2618,7 +2625,8 @@ Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_globa physical_plan ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] -----MemoryExec: partitions=1, partition_sizes=[1] +----SortExec: expr=[amount@2 ASC NULLS LAST] +------MemoryExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2652,7 +2660,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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] +----SortExec: expr=[ts@1 DESC] +------MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2685,7 +2694,8 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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] +----SortExec: expr=[ts@1 DESC] +------MemoryExec: partitions=1, partition_sizes=[1] query TRRR SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2721,11 +2731,12 @@ physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] --ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] ----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] -------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] ---------CoalesceBatchesExec: target_batch_size=8192 -----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 -------------MemoryExec: partitions=1, partition_sizes=[1] -------------MemoryExec: partitions=1, partition_sizes=[1] +------SortExec: expr=[sn@5 ASC NULLS LAST] +--------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] +----------CoalesceBatchesExec: target_batch_size=8192 +------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 +--------------MemoryExec: partitions=1, partition_sizes=[1] +--------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2770,7 +2781,8 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------------SortExec: expr=[ts@1 ASC NULLS LAST] +------------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2806,7 +2818,8 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] -----------------MemoryExec: partitions=1, partition_sizes=[1] +----------------SortExec: expr=[ts@1 ASC NULLS LAST] +------------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2840,8 +2853,9 @@ ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +--------SortExec: expr=[ts@0 ASC NULLS LAST] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2866,8 +2880,9 @@ ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +--------SortExec: expr=[ts@0 ASC NULLS LAST] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2890,8 +2905,9 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts A --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +--------SortExec: expr=[ts@0 ASC NULLS LAST] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -2913,8 +2929,9 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts D --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +--------SortExec: expr=[ts@0 DESC] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -2936,8 +2953,9 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amou --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------MemoryExec: partitions=1, partition_sizes=[1] +--------SortExec: expr=[amount@0 ASC NULLS LAST] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -2965,8 +2983,9 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +--------------SortExec: expr=[amount@1 ASC NULLS LAST] +----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------------MemoryExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3000,8 +3019,9 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------MemoryExec: partitions=1, partition_sizes=[1] +--------------SortExec: expr=[amount@1 DESC] +----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3630,8 +3650,9 @@ ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_o ----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)] -----------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 +----------SortExec: expr=[c@1 DESC] +------------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 query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -4070,13 +4091,14 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +----------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC] +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; -query II?? +query II?? rowsort SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table GROUP BY a, b @@ -4090,7 +4112,7 @@ SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) statement ok set datafusion.execution.target_partitions = 8; -query II?? +query II?? rowsort SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table GROUP BY a, b @@ -4117,13 +4139,14 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +----------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 DESC] +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; -query IIII? +query IIII? rowsort SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table GROUP BY a, b @@ -4172,13 +4195,14 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=20 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 ----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +------------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC] +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; -query II?? +query II?? rowsort SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table GROUP BY a, b @@ -4220,13 +4244,14 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=20 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 ----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +------------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 DESC] +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; -query IIII? +query IIII? rowsort SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table GROUP BY a, b @@ -4237,7 +4262,7 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRA 1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] -query IIII +query IIII rowsort SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) FROM multiple_ordered_table GROUP BY a, b From 16af57f39ff31bc006e75250761aa08c53180ac3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 15:30:36 +0300 Subject: [PATCH 21/49] Simplifications --- .../src/physical_optimizer/enforce_sorting.rs | 290 +----------------- .../physical-plan/src/aggregates/mod.rs | 280 +++-------------- .../src/aggregates/no_grouping.rs | 9 +- 3 files changed, 60 insertions(+), 519 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index d9212434b927..1c8592bcffc1 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2356,284 +2356,6 @@ WITH HEADER ROW WITH ORDER (a ASC, b ASC, c ASC) LOCATION '../core/tests/data/window_2.csv';"; - // #[tokio::test] - // async fn test_query() -> Result<()> { - // let config = SessionConfig::new().with_target_partitions(1); - // let ctx = SessionContext::new_with_config(config); - // - // ctx.sql(ANNOTATED_DATA_FINITE2).await?; - // - // let sql = - // "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) - // FROM annotated_data_finite2 - // GROUP BY a, b - // ORDER BY a, b"; - // - // // let sql = "SELECT ARRAY_AGG(d ORDER BY c ASC) - // // FROM annotated_data_finite2 - // // GROUP BY a - // // ORDER BY a"; - // - // let msg = format!("Creating logical plan for '{sql}'"); - // let dataframe = ctx.sql(sql).await.expect(&msg); - // let physical_plan = dataframe.create_physical_plan().await?; - // print_plan(&physical_plan)?; - // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&batches)?; - // - // let expected = vec![ - // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } - // - // #[tokio::test] - // async fn test_query2() -> Result<()> { - // let config = SessionConfig::new() - // .with_target_partitions(8) - // .with_batch_size(4); - // let ctx = SessionContext::new_with_config(config); - // - // ctx.sql(SALES_GLOBAL).await?; - // - // let sql = "SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 - // FROM sales_global"; - // - // let msg = format!("Creating logical plan for '{sql}'"); - // let dataframe = ctx.sql(sql).await.expect(&msg); - // let physical_plan = dataframe.create_physical_plan().await?; - // print_plan(&physical_plan)?; - // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&batches)?; - // - // let expected = vec![ - // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } - // - // #[tokio::test] - // async fn test_query3() -> Result<()> { - // let config = SessionConfig::new() - // .with_target_partitions(8) - // .with_batch_size(1000); - // let ctx = SessionContext::new_with_config(config); - // - // ctx.sql(MULTIPLE_ORDERED_TABLE).await?; - // - // let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) - // FROM multiple_ordered_table - // GROUP BY a, b - // ORDER BY a, b"; - // - // let msg = format!("Creating logical plan for '{sql}'"); - // let dataframe = ctx.sql(sql).await.expect(&msg); - // let physical_plan = dataframe.create_physical_plan().await?; - // print_plan(&physical_plan)?; - // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&batches)?; - // - // let expected = vec![ - // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } - // - // #[tokio::test] - // async fn test_query4() -> Result<()> { - // let config = SessionConfig::new() - // .with_target_partitions(8) - // .with_batch_size(1000); - // let ctx = SessionContext::new_with_config(config); - // - // ctx.sql(MULTIPLE_ORDERED_TABLE).await?; - // - // let sql = - // "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) - // FROM multiple_ordered_table - // GROUP BY a, b - // ORDER BY a, b"; - // - // let msg = format!("Creating logical plan for '{sql}'"); - // let dataframe = ctx.sql(sql).await.expect(&msg); - // let physical_plan = dataframe.create_physical_plan().await?; - // print_plan(&physical_plan)?; - // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&batches)?; - // - // let expected = vec![ - // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } - // - // #[tokio::test] - // async fn test_query5() -> Result<()> { - // let config = SessionConfig::new() - // .with_target_partitions(1) - // .with_batch_size(4); - // let ctx = SessionContext::new_with_config(config); - // - // ctx.sql(MULTIPLE_ORDERED_TABLE).await?; - // - // let sql = "SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) - // FROM multiple_ordered_table - // GROUP BY a, b - // ORDER BY a, b"; - // - // let msg = format!("Creating logical plan for '{sql}'"); - // let dataframe = ctx.sql(sql).await.expect(&msg); - // let physical_plan = dataframe.create_physical_plan().await?; - // print_plan(&physical_plan)?; - // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&batches)?; - // - // let expected = vec![ - // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } - // - // #[tokio::test] - // async fn test_query6() -> Result<()> { - // let config = SessionConfig::new() - // .with_target_partitions(8) - // .with_batch_size(1000); - // let ctx = SessionContext::new_with_config(config); - // - // ctx.sql(SALES_GLOBAL).await?; - // - // let sql = "SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, - // LAST_VALUE(amount ORDER BY ts ASC) AS fv2 - // FROM sales_global"; - // - // let msg = format!("Creating logical plan for '{sql}'"); - // let dataframe = ctx.sql(sql).await.expect(&msg); - // let physical_plan = dataframe.create_physical_plan().await?; - // print_plan(&physical_plan)?; - // let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&batches)?; - // - // let expected = vec![ - // "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(annotated_data_finite2.d), ARRAY_AGG(annotated_data_finite2.d)], ordering_mode=Sorted", - // " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - // ]; - // // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); - // - // let expected = [ - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| a | b | ARRAY_AGG(annotated_data_finite2.d) | ARRAY_AGG(annotated_data_finite2.d) |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - // "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - // "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - // "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - // "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - // ]; - // assert_batches_eq!(expected, &batches); - // Ok(()) - // } - #[tokio::test] async fn test_query7() -> Result<()> { let config = SessionConfig::new() @@ -2847,7 +2569,7 @@ GROUP BY d"; let expected = vec![ "ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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=Single, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.c), LAST_VALUE(multiple_ordered_table.c)]", - " SortExec: expr=[a@0 DESC,c@1 DESC]", + " SortExec: expr=[c@1 DESC]", " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", ]; // Get string representation of the plan @@ -2902,9 +2624,10 @@ ORDER BY d"; " 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.c), LAST_VALUE(multiple_ordered_table.c)]", - " SortExec: expr=[a@0 DESC,c@1 DESC]", + " SortExec: expr=[c@1 DESC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", ]; + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", + ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); assert_eq!( @@ -2956,8 +2679,9 @@ ORDER BY d"; " CoalesceBatchesExec: target_batch_size=2", " RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8", " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true", + " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC]", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true", ]; // Get string representation of the plan let actual = get_plan_string(&physical_plan); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 311bc58872a3..a1703dd27439 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -27,9 +27,7 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; -use crate::windows::{ - get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, -}; +use crate::windows::{get_ordered_partition_by_indices, PartitionSearchMode}; use crate::{ DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -46,13 +44,12 @@ use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::collapse_lex_req, expressions::{Column, Max, Min, UnKnownColumn}, - physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, - LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, + LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; -use futures::StreamExt; use hashbrown::HashSet; -use itertools::{izip, Itertools}; +use itertools::Itertools; mod group_values; mod no_grouping; @@ -322,144 +319,6 @@ pub struct AggregateExec { output_ordering: Option, } -/// This function returns the ordering requirement of the first non-reversible -/// order-sensitive aggregate function such as ARRAY_AGG. This requirement serves -/// as the initial requirement while calculating the finest requirement among all -/// aggregate functions. If this function returns `None`, it means there is no -/// hard ordering requirement for the aggregate functions (in terms of direction). -/// Then, we can generate two alternative requirements with opposite directions. -fn get_init_req( - aggr_expr: &[Arc], - order_by_expr: &[Option], -) -> Option { - for (aggr_expr, fn_reqs) in aggr_expr.iter().zip(order_by_expr.iter()) { - // If the aggregation function is a non-reversible order-sensitive function - // and there is a hard requirement, choose first such requirement: - if is_order_sensitive(aggr_expr) - && aggr_expr.reverse_expr().is_none() - && fn_reqs.is_some() - { - return fn_reqs.clone(); - } - } - None -} - -/// This function gets the finest ordering requirement among all the aggregation -/// functions. If requirements are conflicting, (i.e. we can not compute the -/// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement( - aggr_expr: &mut [Arc], - order_by_expr: &mut [Option], - eq_properties: &EquivalenceProperties, -) -> Result> { - return Ok(None); - // First, we check if all the requirements are satisfied by the existing - // ordering. If so, we return `None` to indicate this. - let mut all_satisfied = true; - for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - if eq_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { - continue; - } - if let Some(reverse) = aggr_expr.reverse_expr() { - let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); - if eq_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { - // We need to update `aggr_expr` with its reverse since only its - // reverse requirement is compatible with the existing requirements: - *aggr_expr = reverse; - *fn_req = reverse_req; - continue; - } - } - // Requirement is not satisfied: - all_satisfied = false; - } - if all_satisfied { - // All of the requirements are already satisfied. - return Ok(None); - } - let mut finest_req = get_init_req(aggr_expr, order_by_expr); - for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - let Some(fn_req) = fn_req else { - continue; - }; - - if let Some(finest_req) = &mut finest_req { - if let Some(finer) = eq_properties.get_finer_ordering(finest_req, fn_req) { - *finest_req = finer; - continue; - } - // If an aggregate function is reversible, analyze whether its reverse - // direction is compatible with existing requirements: - if let Some(reverse) = aggr_expr.reverse_expr() { - let fn_req_reverse = reverse_order_bys(fn_req); - if let Some(finer) = - eq_properties.get_finer_ordering(finest_req, &fn_req_reverse) - { - // We need to update `aggr_expr` with its reverse, since only its - // reverse requirement is compatible with existing requirements: - *aggr_expr = reverse; - *finest_req = finer; - *fn_req = fn_req_reverse; - continue; - } - } - // // If neither of the requirements 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" - // ); - return Ok(None); - } else { - finest_req = Some(fn_req.clone()); - } - } - Ok(finest_req) -} - -/// Calculates search_mode for the aggregation -fn get_aggregate_search_mode( - group_by: &PhysicalGroupBy, - input: &Arc, - aggr_expr: &mut [Arc], - order_by_expr: &mut [Option], - ordering_req: &mut Vec, -) -> PartitionSearchMode { - let groupby_exprs = group_by - .expr - .iter() - .map(|(item, _)| item.clone()) - .collect::>(); - let mut partition_search_mode = PartitionSearchMode::Linear; - if !group_by.is_single() || groupby_exprs.is_empty() { - return partition_search_mode; - } - - if let Some((should_reverse, mode)) = - get_window_mode(&groupby_exprs, ordering_req, input) - { - let all_reversible = aggr_expr - .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); - if should_reverse && all_reversible { - izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( - |(aggr, order_by)| { - if let Some(reverse) = aggr.reverse_expr() { - *aggr = reverse; - } else { - unreachable!(); - } - *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); - }, - ); - *ordering_req = reverse_order_bys(ordering_req); - } - partition_search_mode = mode; - } - partition_search_mode -} - /// Check whether group by expression contains all of the expression inside `requirement` // As an example Group By (c,b,a) contains all of the expressions in the `requirement`: (a ASC, b DESC) fn group_by_contains_all_requirements( @@ -485,7 +344,7 @@ impl AggregateExec { mut aggr_expr: Vec>, filter_expr: Vec>>, // Ordering requirement of each aggregate expression - mut order_by_expr: Vec>, + order_by_expr: Vec>, input: Arc, input_schema: SchemaRef, ) -> Result { @@ -501,42 +360,6 @@ impl AggregateExec { get_groups_indices(&mut aggr_expr, &group_by, &eq_properties, &mode); let schema = Arc::new(schema); - // // Reset ordering requirement to `None` if aggregator is not order-sensitive - // order_by_expr = aggr_expr - // .iter() - // .zip(order_by_expr) - // .map(|(aggr_expr, fn_reqs)| { - // // If - // // - aggregation function is order-sensitive and - // // - aggregation is performing a "first stage" calculation, and - // // - at least one of the aggregate function requirement is not inside group by expression - // // keep the ordering requirement as is; otherwise ignore the ordering requirement. - // // 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. - // fn_reqs.filter(|req| { - // is_order_sensitive(aggr_expr) - // && mode.is_first_stage() - // && !group_by_contains_all_requirements(&group_by, req) - // }) - // }) - // .collect::>(); - // let requirement = get_finest_requirement( - // &mut aggr_expr, - // &mut order_by_expr, - // &input.equivalence_properties(), - // )?; - // let mut ordering_req = requirement.unwrap_or(vec![]); - // let partition_search_mode = get_aggregate_search_mode( - // &group_by, - // &input, - // &mut aggr_expr, - // &mut order_by_expr, - // &mut ordering_req, - // ); let mut ordering_req = vec![]; if let Some(max_group) = aggregate_groups @@ -567,9 +390,9 @@ impl AggregateExec { new_requirement = collapse_lex_req(new_requirement); let partition_search_mode = - if indices.len() == groupby_exprs.len() && !groupby_exprs.is_empty() { + if indices.len() == groupby_exprs.len() && !indices.is_empty() { PartitionSearchMode::Sorted - } else if indices.len() > 0 { + } else if !indices.is_empty() { PartitionSearchMode::PartiallySorted(indices) } else { PartitionSearchMode::Linear @@ -1027,9 +850,19 @@ fn get_req( agg_mode: &AggregateMode, ) -> LexOrdering { let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); + // If + // - aggregation function is not order-sensitive and + // - aggregation is performing a "second stage" calculation, and + // - all aggregate function requirement is inside group by expression + // ignore the ordering requirement. + // 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 !is_order_sensitive(aggr_expr) - || group_by_contains_all_requirements(&group_by, &req) - // For final stages there is no requirement + || group_by_contains_all_requirements(group_by, &req) || !agg_mode.is_first_stage() { req.clear(); @@ -1043,25 +876,17 @@ fn get_groups_indices( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Vec { - let mut initial_groups = vec![]; - for idx in 0..aggr_exprs.len() { - let aggr_expr = &aggr_exprs[idx]; - let req = get_req(&aggr_expr, group_by, agg_mode); - initial_groups.push((vec![idx], req)); - } - let mut used_indices: HashSet = HashSet::new(); let mut groups = vec![]; while used_indices.len() != aggr_exprs.len() { let mut group: Option<(Vec, LexOrdering)> = None; - for idx in 0..aggr_exprs.len() { - let aggr_expr = &mut aggr_exprs[idx]; + for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { // Group is empty and index is not already in another group. if used_indices.contains(&idx) { // Skip this group, it is already inserted. continue; } - let aggr_req = get_req(&aggr_expr, group_by, agg_mode); + let aggr_req = get_req(aggr_expr, group_by, agg_mode); if let Some((group_indices, req)) = &mut group { if let Some(finer) = eq_properties.get_finer_ordering(req, &aggr_req) { *req = finer; @@ -1181,10 +1006,7 @@ fn finalize_aggregation_groups( .collect::>>()?; // TODO: Add proper indices // Convert Vec> to Vec. - let res = elems - .into_iter() - .flat_map(|elems| elems) - .collect::>(); + let res = elems.into_iter().flatten().collect::>(); Ok(res) } @@ -1319,9 +1141,7 @@ mod tests { use std::task::{Context, Poll}; use super::*; - use crate::aggregates::{ - get_finest_requirement, AggregateExec, AggregateMode, PhysicalGroupBy, - }; + use crate::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common; @@ -1353,7 +1173,6 @@ mod tests { use datafusion_execution::memory_pool::FairSpillPool; use futures::{FutureExt, Stream}; - use itertools::GroupBy; // Generate a schema which consists of 5 columns (a, b, c, d, e) fn create_test_schema() -> Result { @@ -1380,16 +1199,6 @@ mod tests { .collect::>() } - // Convert each inner tuple to PhysicalSortExpr - fn convert_to_orderings( - orderings: &[Vec<(&Arc, SortOptions)>], - ) -> Vec> { - orderings - .iter() - .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) - .collect() - } - /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. @@ -2254,7 +2063,7 @@ mod tests { 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 mut order_by_exprs = vec![ + let order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { expr: col_a.clone(), @@ -2291,7 +2100,7 @@ mod tests { options: options2, }]), ]; - let common_requirement = Some(vec![ + let common_requirement = vec![ PhysicalSortExpr { expr: col_a.clone(), options: options1, @@ -2300,17 +2109,29 @@ mod tests { expr: col_c.clone(), options: options1, }, - ]); - let aggr_expr = Arc::new(FirstValue::new( - col_a.clone(), - "first1", - DataType::Int32, - vec![], - vec![], - )) as _; - let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = - get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; + ]; + let mut aggr_exprs = order_by_exprs + .into_iter() + .map(|order_by_expr| { + Arc::new(FirstValue::new( + col_a.clone(), + "first1", + DataType::Int32, + order_by_expr.unwrap_or_default(), + vec![], + )) as _ + }) + .collect::>(); + // let res = + // get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; + let group_by = PhysicalGroupBy::new_single(vec![]); + let res = get_groups_indices( + &mut aggr_exprs, + &group_by, + &eq_properties, + &AggregateMode::Partial, + ); + let res = res[0].requirement.clone(); assert_eq!(res, common_requirement); Ok(()) } @@ -2381,14 +2202,13 @@ mod tests { .into_iter() .map(|req| { let req = convert_to_sort_exprs(&req); - let aggr_expr = Arc::new(FirstValue::new( + Arc::new(FirstValue::new( col_a.clone(), "first1", DataType::Int32, req, vec![], - )) as _; - aggr_expr + )) as _ }) .collect::>(); diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 4301a1b559ad..967aec25d65c 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -18,17 +18,15 @@ //! Aggregate without grouping columns use crate::aggregates::{ - aggregate_expressions, create_accumulators, finalize_aggregation, - finalize_aggregation_groups, AccumulatorItem, AggregateExprGroup, AggregateGroup, - AggregateMode, + aggregate_expressions, create_accumulators, finalize_aggregation_groups, + AggregateExprGroup, AggregateGroup, AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; -use crate::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; +use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::PhysicalExpr; use futures::stream::BoxStream; use std::borrow::Cow; use std::sync::Arc; @@ -95,7 +93,6 @@ impl AggregateStream { vec![None; agg.aggr_expr.len()] } }; - let accumulators = create_accumulators(&agg.aggr_expr)?; let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); From d442e8496513713487afa320630c97539a5aa9c6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 16:29:42 +0300 Subject: [PATCH 22/49] Resolve linter errors --- .../src/physical_optimizer/enforce_sorting.rs | 848 +++++++++--------- .../physical-plan/src/aggregates/row_hash.rs | 6 +- 2 files changed, 426 insertions(+), 428 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 1c8592bcffc1..1f6d88abde1a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2280,427 +2280,427 @@ mod tests { } } -#[cfg(test)] -mod tmp_tests { - use crate::assert_batches_eq; - use crate::physical_plan::{collect, displayable, ExecutionPlan}; - use crate::prelude::SessionContext; - use arrow::util::pretty::print_batches; - use datafusion_common::Result; - use datafusion_execution::config::SessionConfig; - use datafusion_physical_plan::get_plan_string; - use std::sync::Arc; - - fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) - } - - const MULTIPLE_ORDERED_TABLE: &str = "CREATE EXTERNAL TABLE multiple_ordered_table ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW -WITH ORDER (a ASC, b ASC) -WITH ORDER (c ASC) - LOCATION 'tests/data/window_2.csv'"; - const ANNOTATED_DATA_FINITE2: &str = "CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION 'tests/data/window_2.csv'"; - - const SALES_GLOBAL: &str = "CREATE TABLE sales_global (zip_code INT, - country VARCHAR(3), - sn INT, - ts TIMESTAMP, - currency VARCHAR(3), - amount FLOAT - ) as VALUES - (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), - (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), - (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), - (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), - (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), - (0, 'GRC', 4, '2022-01-03 11:00:00'::timestamp, 'EUR', 80.0)"; - - const T1: &str = - "CREATE TABLE tab1(col0 INTEGER, col1 INTEGER, col2 INTEGER) as VALUES - (22,6,8), - (28,57,45), - (82,44,71) - "; - - const ANNOTATED_DATA_INFINITE2: &str = - "CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER -) -STORED AS CSV -WITH HEADER ROW -WITH ORDER (a ASC, b ASC, c ASC) -LOCATION '../core/tests/data/window_2.csv';"; - - #[tokio::test] - async fn test_query7() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(1) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(SALES_GLOBAL).await?; - - let sql = "SELECT - FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, - LAST_VALUE(amount ORDER BY ts ASC) AS lv1, - FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, - LAST_VALUE(amount ORDER BY ts DESC) AS lv2 - FROM sales_global"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as lv2]", - " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", - " SortExec: expr=[ts@0 ASC NULLS LAST]", - " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+------+------+------+------+", - "| fv1 | lv1 | fv2 | lv2 |", - "+------+------+------+------+", - "| 30.0 | 80.0 | 80.0 | 30.0 |", - "+------+------+------+------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query8() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(1) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(SALES_GLOBAL).await?; - - let sql = "SELECT - FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, - LAST_VALUE(amount ORDER BY ts ASC) AS lv1, - FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, - LAST_VALUE(amount ORDER BY ts DESC) AS lv2 - FROM sales_global - GROUP BY country - ORDER BY country"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "ProjectionExec: expr=[fv1@0 as fv1, lv1@1 as lv1, fv2@2 as fv2, lv2@3 as lv2]", - " SortExec: expr=[country@4 ASC NULLS LAST]", - " ProjectionExec: expr=[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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@4 as lv2, country@0 as country]", - " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", - " SortExec: expr=[ts@1 ASC NULLS LAST]", - " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+------+-------+-------+------+", - "| fv1 | lv1 | fv2 | lv2 |", - "+------+-------+-------+------+", - "| 50.0 | 200.0 | 200.0 | 50.0 |", - "| 30.0 | 80.0 | 80.0 | 30.0 |", - "| 75.0 | 100.0 | 100.0 | 75.0 |", - "+------+-------+-------+------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query9() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(1) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(T1).await?; - - let sql = "SELECT - tab1.col0 * 84 + + 38 AS col2 FROM tab1 GROUP BY tab1.col0"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "ProjectionExec: expr=[CAST((- col0@0) AS Int64) * 84 + 38 as col2]", - " AggregateExec: mode=Single, gby=[col0@0 as col0], aggr=[]", - " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+-------+", - "| col2 |", - "+-------+", - "| -1810 |", - "| -2314 |", - "| -6850 |", - "+-------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query10() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(1) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(ANNOTATED_DATA_INFINITE2).await?; - - let sql = "SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c - FROM annotated_data_infinite2 - GROUP BY a, b"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c]", - " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST]", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---+---------+", - "| a | b | first_c |", - "+---+---+---------+", - "| 0 | 0 | 0 |", - "| 0 | 1 | 25 |", - "| 1 | 2 | 50 |", - "| 1 | 3 | 75 |", - "+---+---+---------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query11() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(1) - .with_batch_size(1000); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(MULTIPLE_ORDERED_TABLE).await?; - - let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, - LAST_VALUE(c ORDER BY c DESC) as last_c -FROM multiple_ordered_table -GROUP BY d"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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=Single, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.c), LAST_VALUE(multiple_ordered_table.c)]", - " SortExec: expr=[c@1 DESC]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---------+--------+", - "| d | first_a | last_c |", - "+---+---------+--------+", - "| 3 | 99 | 15 |", - "| 4 | 98 | 9 |", - "| 2 | 97 | 1 |", - "| 0 | 95 | 0 |", - "| 1 | 90 | 4 |", - "+---+---------+--------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query12() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(8) - .with_batch_size(2); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(MULTIPLE_ORDERED_TABLE).await?; - - let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, - LAST_VALUE(c ORDER BY c DESC) as last_c -FROM multiple_ordered_table -GROUP BY d -ORDER BY d"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "SortPreservingMergeExec: [d@0 ASC NULLS LAST]", - " SortExec: expr=[d@0 ASC NULLS LAST]", - " ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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.c), 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.c), LAST_VALUE(multiple_ordered_table.c)]", - " SortExec: expr=[c@1 DESC]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---------+--------+", - "| d | first_a | last_c |", - "+---+---------+--------+", - "| 0 | 95 | 0 |", - "| 1 | 90 | 4 |", - "| 2 | 97 | 1 |", - "| 3 | 99 | 15 |", - "| 4 | 98 | 9 |", - "+---+---------+--------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } - - #[tokio::test] - async fn test_query13() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(8) - .with_batch_size(2); - let ctx = SessionContext::new_with_config(config); - - ctx.sql(MULTIPLE_ORDERED_TABLE).await?; - - let sql = - "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) - FROM multiple_ordered_table - GROUP BY a, b - ORDER BY a, b"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&batches)?; - - let expected = vec![ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", - " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", - " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=2", - " RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8", - " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", - " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let expected = [ - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| a | b | ARRAY_AGG(multiple_ordered_table.d) | ARRAY_AGG(multiple_ordered_table.d) |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", - "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", - "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", - "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", - "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", - ]; - assert_batches_eq!(expected, &batches); - Ok(()) - } -} +// #[cfg(test)] +// mod tmp_tests { +// use crate::assert_batches_eq; +// use crate::physical_plan::{collect, displayable, ExecutionPlan}; +// use crate::prelude::SessionContext; +// use arrow::util::pretty::print_batches; +// use datafusion_common::Result; +// use datafusion_execution::config::SessionConfig; +// use datafusion_physical_plan::get_plan_string; +// use std::sync::Arc; +// +// fn print_plan(plan: &Arc) -> Result<()> { +// let formatted = displayable(plan.as_ref()).indent(true).to_string(); +// let actual: Vec<&str> = formatted.trim().lines().collect(); +// println!("{:#?}", actual); +// Ok(()) +// } +// +// const MULTIPLE_ORDERED_TABLE: &str = "CREATE EXTERNAL TABLE multiple_ordered_table ( +// a0 INTEGER, +// a INTEGER, +// b INTEGER, +// c INTEGER, +// d INTEGER +// ) +// STORED AS CSV +// WITH HEADER ROW +// WITH ORDER (a ASC, b ASC) +// WITH ORDER (c ASC) +// LOCATION 'tests/data/window_2.csv'"; +// const ANNOTATED_DATA_FINITE2: &str = "CREATE EXTERNAL TABLE annotated_data_finite2 ( +// a0 INTEGER, +// a INTEGER, +// b INTEGER, +// c INTEGER, +// d INTEGER +// ) +// STORED AS CSV +// WITH HEADER ROW +// WITH ORDER (a ASC, b ASC, c ASC) +// LOCATION 'tests/data/window_2.csv'"; +// +// const SALES_GLOBAL: &str = "CREATE TABLE sales_global (zip_code INT, +// country VARCHAR(3), +// sn INT, +// ts TIMESTAMP, +// currency VARCHAR(3), +// amount FLOAT +// ) as VALUES +// (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), +// (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), +// (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), +// (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), +// (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), +// (0, 'GRC', 4, '2022-01-03 11:00:00'::timestamp, 'EUR', 80.0)"; +// +// const T1: &str = +// "CREATE TABLE tab1(col0 INTEGER, col1 INTEGER, col2 INTEGER) as VALUES +// (22,6,8), +// (28,57,45), +// (82,44,71) +// "; +// +// const ANNOTATED_DATA_INFINITE2: &str = +// "CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( +// a0 INTEGER, +// a INTEGER, +// b INTEGER, +// c INTEGER, +// d INTEGER +// ) +// STORED AS CSV +// WITH HEADER ROW +// WITH ORDER (a ASC, b ASC, c ASC) +// LOCATION '../core/tests/data/window_2.csv';"; +// +// #[tokio::test] +// async fn test_query7() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(1) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(SALES_GLOBAL).await?; +// +// let sql = "SELECT +// FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, +// LAST_VALUE(amount ORDER BY ts ASC) AS lv1, +// FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, +// LAST_VALUE(amount ORDER BY ts DESC) AS lv2 +// FROM sales_global"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as lv2]", +// " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", +// " SortExec: expr=[ts@0 ASC NULLS LAST]", +// " MemoryExec: partitions=1, partition_sizes=[1]", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+------+------+------+------+", +// "| fv1 | lv1 | fv2 | lv2 |", +// "+------+------+------+------+", +// "| 30.0 | 80.0 | 80.0 | 30.0 |", +// "+------+------+------+------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query8() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(1) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(SALES_GLOBAL).await?; +// +// let sql = "SELECT +// FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, +// LAST_VALUE(amount ORDER BY ts ASC) AS lv1, +// FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, +// LAST_VALUE(amount ORDER BY ts DESC) AS lv2 +// FROM sales_global +// GROUP BY country +// ORDER BY country"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[fv1@0 as fv1, lv1@1 as lv1, fv2@2 as fv2, lv2@3 as lv2]", +// " SortExec: expr=[country@4 ASC NULLS LAST]", +// " ProjectionExec: expr=[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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@4 as lv2, country@0 as country]", +// " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", +// " SortExec: expr=[ts@1 ASC NULLS LAST]", +// " MemoryExec: partitions=1, partition_sizes=[1]", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+------+-------+-------+------+", +// "| fv1 | lv1 | fv2 | lv2 |", +// "+------+-------+-------+------+", +// "| 50.0 | 200.0 | 200.0 | 50.0 |", +// "| 30.0 | 80.0 | 80.0 | 30.0 |", +// "| 75.0 | 100.0 | 100.0 | 75.0 |", +// "+------+-------+-------+------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query9() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(1) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(T1).await?; +// +// let sql = "SELECT - tab1.col0 * 84 + + 38 AS col2 FROM tab1 GROUP BY tab1.col0"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[CAST((- col0@0) AS Int64) * 84 + 38 as col2]", +// " AggregateExec: mode=Single, gby=[col0@0 as col0], aggr=[]", +// " MemoryExec: partitions=1, partition_sizes=[1]", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+-------+", +// "| col2 |", +// "+-------+", +// "| -1810 |", +// "| -2314 |", +// "| -6850 |", +// "+-------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query10() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(1) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(ANNOTATED_DATA_INFINITE2).await?; +// +// let sql = "SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c +// FROM annotated_data_infinite2 +// GROUP BY a, b"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c]", +// " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted", +// " StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST]", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+---------+", +// "| a | b | first_c |", +// "+---+---+---------+", +// "| 0 | 0 | 0 |", +// "| 0 | 1 | 25 |", +// "| 1 | 2 | 50 |", +// "| 1 | 3 | 75 |", +// "+---+---+---------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query11() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(1) +// .with_batch_size(1000); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(MULTIPLE_ORDERED_TABLE).await?; +// +// let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, +// LAST_VALUE(c ORDER BY c DESC) as last_c +// FROM multiple_ordered_table +// GROUP BY d"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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=Single, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.c), LAST_VALUE(multiple_ordered_table.c)]", +// " SortExec: expr=[c@1 DESC]", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---------+--------+", +// "| d | first_a | last_c |", +// "+---+---------+--------+", +// "| 3 | 99 | 15 |", +// "| 4 | 98 | 9 |", +// "| 2 | 97 | 1 |", +// "| 0 | 95 | 0 |", +// "| 1 | 90 | 4 |", +// "+---+---------+--------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query12() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(8) +// .with_batch_size(2); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(MULTIPLE_ORDERED_TABLE).await?; +// +// let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, +// LAST_VALUE(c ORDER BY c DESC) as last_c +// FROM multiple_ordered_table +// GROUP BY d +// ORDER BY d"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "SortPreservingMergeExec: [d@0 ASC NULLS LAST]", +// " SortExec: expr=[d@0 ASC NULLS LAST]", +// " ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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.c), 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.c), LAST_VALUE(multiple_ordered_table.c)]", +// " SortExec: expr=[c@1 DESC]", +// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---------+--------+", +// "| d | first_a | last_c |", +// "+---+---------+--------+", +// "| 0 | 95 | 0 |", +// "| 1 | 90 | 4 |", +// "| 2 | 97 | 1 |", +// "| 3 | 99 | 15 |", +// "| 4 | 98 | 9 |", +// "+---+---------+--------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// +// #[tokio::test] +// async fn test_query13() -> Result<()> { +// let config = SessionConfig::new() +// .with_target_partitions(8) +// .with_batch_size(2); +// let ctx = SessionContext::new_with_config(config); +// +// ctx.sql(MULTIPLE_ORDERED_TABLE).await?; +// +// let sql = +// "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) +// FROM multiple_ordered_table +// GROUP BY a, b +// ORDER BY a, b"; +// +// let msg = format!("Creating logical plan for '{sql}'"); +// let dataframe = ctx.sql(sql).await.expect(&msg); +// let physical_plan = dataframe.create_physical_plan().await?; +// print_plan(&physical_plan)?; +// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; +// print_batches(&batches)?; +// +// let expected = vec![ +// "SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", +// " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", +// " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", +// " CoalesceBatchesExec: target_batch_size=2", +// " RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8", +// " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", +// " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC]", +// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", +// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true", +// ]; +// // Get string representation of the plan +// let actual = get_plan_string(&physical_plan); +// assert_eq!( +// expected, actual, +// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" +// ); +// +// let expected = [ +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| a | b | ARRAY_AGG(multiple_ordered_table.d) | ARRAY_AGG(multiple_ordered_table.d) |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", +// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", +// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", +// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", +// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", +// ]; +// assert_batches_eq!(expected, &batches); +// Ok(()) +// } +// } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 2272ecb7ad8c..62c4cb38b769 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -86,9 +86,6 @@ struct SpillState { /// true when streaming merge is in progress is_stream_merging: bool, - /// aggregate_arguments for merging spilled data - merging_aggregate_arguments: Vec>>, - /// GROUP BY expressions for merging spilled data merging_group_by: PhysicalGroupBy, } @@ -396,7 +393,6 @@ impl GroupedHashAggregateStream { spill_expr, spill_schema: agg_schema.clone(), is_stream_merging: false, - merging_aggregate_arguments, merging_group_by: PhysicalGroupBy::new_single(agg_group_by.expr.clone()), }; @@ -437,6 +433,8 @@ pub struct HashAggregateGroup { /// some aggregates such as `CORR` can accept more than one /// argument. aggregate_arguments: Vec>>, + + /// aggregate_arguments at the input of the merging (may have additional fields) merging_aggregate_arguments: Vec>>, /// Optional filter expression to evaluate, one for each for From 8c7b3164e2dee41ee0b916ea5ed4d5def9458a03 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 16:59:49 +0300 Subject: [PATCH 23/49] Minor changes --- .../src/aggregate/array_agg_ordered.rs | 32 +++++++------------ 1 file changed, 12 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index b8c461f42443..4bd978c7a4a7 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -194,8 +194,18 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { } let value = &values[0]; let orderings = &values[1..]; - let (new_values, new_ordering_values) = - Self::convert_arr_to_vec(value, orderings)?; + + let n_row = value.len(); + // Convert &[ArrayRef] to Vec> + let new_ordering_values = (0..n_row) + .map(|idx| get_row_at_idx(orderings, idx)) + .collect::>>()?; + + // Convert ArrayRef to Vec + let new_values = (0..n_row) + .map(|idx| ScalarValue::try_from_array(value, idx)) + .collect::>>()?; + let sort_options = get_sort_options(&self.ordering_req); // Merge new values and new orderings @@ -334,24 +344,6 @@ impl OrderSensitiveArrayAggAccumulator { let arr = ScalarValue::new_list(&orderings, &struct_type); Ok(ScalarValue::List(arr)) } - - fn convert_arr_to_vec( - value: &ArrayRef, - orderings: &[ArrayRef], - ) -> Result<(Vec, Vec>)> { - let n_row = value.len(); - // Convert &[ArrayRef] to Vec> - let orderings = (0..n_row) - .map(|idx| get_row_at_idx(orderings, idx)) - .collect::>>()?; - - // Convert ArrayRef to Vec - let value = (0..n_row) - .map(|idx| ScalarValue::try_from_array(value, idx)) - .collect::>>()?; - - Ok((value, orderings)) - } } /// This is a wrapper struct to be able to correctly merge ARRAY_AGG From b4003dab9e888d2bea73e48b5f926d94dddae0d8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 5 Dec 2023 17:06:53 +0300 Subject: [PATCH 24/49] Minor changes --- datafusion/physical-expr/src/aggregate/first_last.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 15f2d580378e..24cc32e752d1 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -552,6 +552,7 @@ fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { } /// Gets either first, or last value index inside values columns according to ordering requirements +/// Assumes values is ordered according to ordering_req already. fn get_value_idx( values: &[ArrayRef], ordering_req: LexOrderingRef, From 69e5f5998796e9f649b8967d7c615866ed07fc31 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 09:03:58 +0300 Subject: [PATCH 25/49] Update comments, cleanup --- .../src/physical_optimizer/enforce_sorting.rs | 425 ------------------ .../physical-expr/src/aggregate/first_last.rs | 15 +- .../physical-plan/src/aggregates/mod.rs | 25 -- .../src/aggregates/no_grouping.rs | 37 +- 4 files changed, 40 insertions(+), 462 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 1f6d88abde1a..ff052b5f040c 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2279,428 +2279,3 @@ mod tests { Ok(()) } } - -// #[cfg(test)] -// mod tmp_tests { -// use crate::assert_batches_eq; -// use crate::physical_plan::{collect, displayable, ExecutionPlan}; -// use crate::prelude::SessionContext; -// use arrow::util::pretty::print_batches; -// use datafusion_common::Result; -// use datafusion_execution::config::SessionConfig; -// use datafusion_physical_plan::get_plan_string; -// use std::sync::Arc; -// -// fn print_plan(plan: &Arc) -> Result<()> { -// let formatted = displayable(plan.as_ref()).indent(true).to_string(); -// let actual: Vec<&str> = formatted.trim().lines().collect(); -// println!("{:#?}", actual); -// Ok(()) -// } -// -// const MULTIPLE_ORDERED_TABLE: &str = "CREATE EXTERNAL TABLE multiple_ordered_table ( -// a0 INTEGER, -// a INTEGER, -// b INTEGER, -// c INTEGER, -// d INTEGER -// ) -// STORED AS CSV -// WITH HEADER ROW -// WITH ORDER (a ASC, b ASC) -// WITH ORDER (c ASC) -// LOCATION 'tests/data/window_2.csv'"; -// const ANNOTATED_DATA_FINITE2: &str = "CREATE EXTERNAL TABLE annotated_data_finite2 ( -// a0 INTEGER, -// a INTEGER, -// b INTEGER, -// c INTEGER, -// d INTEGER -// ) -// STORED AS CSV -// WITH HEADER ROW -// WITH ORDER (a ASC, b ASC, c ASC) -// LOCATION 'tests/data/window_2.csv'"; -// -// const SALES_GLOBAL: &str = "CREATE TABLE sales_global (zip_code INT, -// country VARCHAR(3), -// sn INT, -// ts TIMESTAMP, -// currency VARCHAR(3), -// amount FLOAT -// ) as VALUES -// (0, 'GRC', 0, '2022-01-01 06:00:00'::timestamp, 'EUR', 30.0), -// (1, 'FRA', 1, '2022-01-01 08:00:00'::timestamp, 'EUR', 50.0), -// (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), -// (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), -// (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), -// (0, 'GRC', 4, '2022-01-03 11:00:00'::timestamp, 'EUR', 80.0)"; -// -// const T1: &str = -// "CREATE TABLE tab1(col0 INTEGER, col1 INTEGER, col2 INTEGER) as VALUES -// (22,6,8), -// (28,57,45), -// (82,44,71) -// "; -// -// const ANNOTATED_DATA_INFINITE2: &str = -// "CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( -// a0 INTEGER, -// a INTEGER, -// b INTEGER, -// c INTEGER, -// d INTEGER -// ) -// STORED AS CSV -// WITH HEADER ROW -// WITH ORDER (a ASC, b ASC, c ASC) -// LOCATION '../core/tests/data/window_2.csv';"; -// -// #[tokio::test] -// async fn test_query7() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(1) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(SALES_GLOBAL).await?; -// -// let sql = "SELECT -// FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, -// LAST_VALUE(amount ORDER BY ts ASC) AS lv1, -// FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, -// LAST_VALUE(amount ORDER BY ts DESC) AS lv2 -// FROM sales_global"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@2 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as lv2]", -// " AggregateExec: mode=Single, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", -// " SortExec: expr=[ts@0 ASC NULLS LAST]", -// " MemoryExec: partitions=1, partition_sizes=[1]", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+------+------+------+------+", -// "| fv1 | lv1 | fv2 | lv2 |", -// "+------+------+------+------+", -// "| 30.0 | 80.0 | 80.0 | 30.0 |", -// "+------+------+------+------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query8() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(1) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(SALES_GLOBAL).await?; -// -// let sql = "SELECT -// FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, -// LAST_VALUE(amount ORDER BY ts ASC) AS lv1, -// FIRST_VALUE(amount ORDER BY ts DESC) AS fv2, -// LAST_VALUE(amount ORDER BY ts DESC) AS lv2 -// FROM sales_global -// GROUP BY country -// ORDER BY country"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[fv1@0 as fv1, lv1@1 as lv1, fv2@2 as fv2, lv2@3 as lv2]", -// " SortExec: expr=[country@4 ASC NULLS LAST]", -// " ProjectionExec: expr=[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 lv1, FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@3 as fv2, LAST_VALUE(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@4 as lv2, country@0 as country]", -// " AggregateExec: mode=Single, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)]", -// " SortExec: expr=[ts@1 ASC NULLS LAST]", -// " MemoryExec: partitions=1, partition_sizes=[1]", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+------+-------+-------+------+", -// "| fv1 | lv1 | fv2 | lv2 |", -// "+------+-------+-------+------+", -// "| 50.0 | 200.0 | 200.0 | 50.0 |", -// "| 30.0 | 80.0 | 80.0 | 30.0 |", -// "| 75.0 | 100.0 | 100.0 | 75.0 |", -// "+------+-------+-------+------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query9() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(1) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(T1).await?; -// -// let sql = "SELECT - tab1.col0 * 84 + + 38 AS col2 FROM tab1 GROUP BY tab1.col0"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[CAST((- col0@0) AS Int64) * 84 + 38 as col2]", -// " AggregateExec: mode=Single, gby=[col0@0 as col0], aggr=[]", -// " MemoryExec: partitions=1, partition_sizes=[1]", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+-------+", -// "| col2 |", -// "+-------+", -// "| -1810 |", -// "| -2314 |", -// "| -6850 |", -// "+-------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query10() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(1) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(ANNOTATED_DATA_INFINITE2).await?; -// -// let sql = "SELECT a, b, FIRST_VALUE(c ORDER BY a DESC) as first_c -// FROM annotated_data_infinite2 -// GROUP BY a, b"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c]", -// " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted", -// " StreamingTableExec: partition_sizes=1, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST]", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+---------+", -// "| a | b | first_c |", -// "+---+---+---------+", -// "| 0 | 0 | 0 |", -// "| 0 | 1 | 25 |", -// "| 1 | 2 | 50 |", -// "| 1 | 3 | 75 |", -// "+---+---+---------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query11() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(1) -// .with_batch_size(1000); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(MULTIPLE_ORDERED_TABLE).await?; -// -// let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, -// LAST_VALUE(c ORDER BY c DESC) as last_c -// FROM multiple_ordered_table -// GROUP BY d"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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=Single, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.c), LAST_VALUE(multiple_ordered_table.c)]", -// " SortExec: expr=[c@1 DESC]", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---------+--------+", -// "| d | first_a | last_c |", -// "+---+---------+--------+", -// "| 3 | 99 | 15 |", -// "| 4 | 98 | 9 |", -// "| 2 | 97 | 1 |", -// "| 0 | 95 | 0 |", -// "| 1 | 90 | 4 |", -// "+---+---------+--------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query12() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(8) -// .with_batch_size(2); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(MULTIPLE_ORDERED_TABLE).await?; -// -// let sql = "SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, -// LAST_VALUE(c ORDER BY c DESC) as last_c -// FROM multiple_ordered_table -// GROUP BY d -// ORDER BY d"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "SortPreservingMergeExec: [d@0 ASC NULLS LAST]", -// " SortExec: expr=[d@0 ASC NULLS LAST]", -// " ProjectionExec: expr=[d@0 as d, FIRST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.a DESC NULLS FIRST, multiple_ordered_table.c DESC NULLS FIRST]@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.c), 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.c), LAST_VALUE(multiple_ordered_table.c)]", -// " SortExec: expr=[c@1 DESC]", -// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/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", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---------+--------+", -// "| d | first_a | last_c |", -// "+---+---------+--------+", -// "| 0 | 95 | 0 |", -// "| 1 | 90 | 4 |", -// "| 2 | 97 | 1 |", -// "| 3 | 99 | 15 |", -// "| 4 | 98 | 9 |", -// "+---+---------+--------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// -// #[tokio::test] -// async fn test_query13() -> Result<()> { -// let config = SessionConfig::new() -// .with_target_partitions(8) -// .with_batch_size(2); -// let ctx = SessionContext::new_with_config(config); -// -// ctx.sql(MULTIPLE_ORDERED_TABLE).await?; -// -// let sql = -// "SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) -// FROM multiple_ordered_table -// GROUP BY a, b -// ORDER BY a, b"; -// -// let msg = format!("Creating logical plan for '{sql}'"); -// let dataframe = ctx.sql(sql).await.expect(&msg); -// let physical_plan = dataframe.create_physical_plan().await?; -// print_plan(&physical_plan)?; -// let batches = collect(physical_plan.clone(), ctx.task_ctx()).await?; -// print_batches(&batches)?; -// -// let expected = vec![ -// "SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", -// " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", -// " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST]", -// " CoalesceBatchesExec: target_batch_size=2", -// " RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8", -// " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted", -// " SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC]", -// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", -// " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true", -// ]; -// // Get string representation of the plan -// let actual = get_plan_string(&physical_plan); -// assert_eq!( -// expected, actual, -// "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" -// ); -// -// let expected = [ -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| a | b | ARRAY_AGG(multiple_ordered_table.d) | ARRAY_AGG(multiple_ordered_table.d) |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// "| 0 | 0 | [4, 0, 0, 3, 0, 4, 1, 2, 3, 3, 2, 1, 2, 2, 4, 4, 1, 2, 0, 1, 1, 0, 0, 2, 0] | [4, 4, 4, 4, 3, 3, 3, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0, 0] |", -// "| 0 | 1 | [3, 1, 1, 0, 0, 3, 2, 3, 3, 1, 1, 0, 4, 1, 0, 4, 2, 2, 4, 3, 1, 1, 0, 2, 0] | [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] |", -// "| 1 | 2 | [4, 1, 1, 2, 0, 1, 1, 4, 2, 0, 4, 4, 1, 3, 4, 2, 1, 1, 1, 2, 4, 1, 1, 3, 0] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] |", -// "| 1 | 3 | [3, 4, 2, 4, 0, 2, 0, 0, 2, 1, 4, 3, 4, 1, 1, 1, 2, 4, 0, 2, 4, 1, 2, 0, 2] | [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] |", -// "+---+---+-----------------------------------------------------------------------------+-----------------------------------------------------------------------------+", -// ]; -// assert_batches_eq!(expected, &batches); -// Ok(()) -// } -// } diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 24cc32e752d1..1efd61505636 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -551,8 +551,19 @@ fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { .collect::>() } -/// Gets either first, or last value index inside values columns according to ordering requirements -/// Assumes values is ordered according to ordering_req already. +/// Gets either first, or last value index inside `values` batch according to ordering requirements +/// Assumes `values` batch is ordered according to ordering_req already. +/// +/// # Parameters +/// +/// - `values`: A slice of `ArrayRef` representing the values to be processed. (Columns of record batch) +/// - `ordering_req`: A lexical ordering reference specifying the required ordering of values. +/// - `is_set`: Whether any value is stored in the state for `first value` or `last value` (At the beginning this is false.). +/// +/// # Returns +/// +/// A `Result` containing an `Option`. If successful, the `Option` holds the index of the +/// desired value. Returns `None` to indicate no existing value doesn't need to be updated. fn get_value_idx( values: &[ArrayRef], ordering_req: LexOrderingRef, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a1703dd27439..ff2cadc2081e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -268,13 +268,6 @@ impl From for SendableRecordBatchStream { } } -pub struct AggregateGroup { - aggregate_expressions: Vec>>, - filter_expressions: Vec>>, - accumulators: Vec, - requirement: LexOrdering, -} - #[derive(Clone, Debug, PartialEq)] pub struct AggregateExprGroup { /// Aggregate expressions indices @@ -992,24 +985,6 @@ fn create_accumulators( .collect::>>() } -/// returns a vector of ArrayRefs, where each entry corresponds to either the -/// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) -fn finalize_aggregation_groups( - // accumulators: &[AccumulatorItem], - aggregate_groups: &[AggregateGroup], - // TODO: Use Mapping indices - mode: &AggregateMode, -) -> Result> { - let elems = aggregate_groups - .iter() - .map(|aggregate_group| finalize_aggregation(&aggregate_group.accumulators, mode)) - .collect::>>()?; - // TODO: Add proper indices - // Convert Vec> to Vec. - let res = elems.into_iter().flatten().collect::>(); - Ok(res) -} - /// returns a vector of ArrayRefs, where each entry corresponds to either the /// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) fn finalize_aggregation( diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 967aec25d65c..5becd1389861 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -18,13 +18,14 @@ //! Aggregate without grouping columns use crate::aggregates::{ - aggregate_expressions, create_accumulators, finalize_aggregation_groups, - AggregateExprGroup, AggregateGroup, AggregateMode, + aggregate_expressions, create_accumulators, finalize_aggregation, AccumulatorItem, + AggregateExprGroup, AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; +use arrow_array::ArrayRef; use datafusion_common::Result; use datafusion_execution::TaskContext; use futures::stream::BoxStream; @@ -36,6 +37,7 @@ use crate::filter::batch_filter; use crate::sorts::sort::sort_batch; use datafusion_common::utils::get_at_indices; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; use futures::stream::{Stream, StreamExt}; use super::AggregateExec; @@ -58,19 +60,17 @@ struct AggregateStreamInner { mode: AggregateMode, input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, - // aggregate_expressions: Vec>>, - // filter_expressions: Vec>>, - // accumulators: Vec, aggregate_groups: Vec, reservation: MemoryReservation, finished: bool, } -// struct AggregateGroup{ -// aggregate_expressions: Vec>>, -// filter_expressions: Vec>>, -// accumulators: Vec, -// } +pub struct AggregateGroup { + aggregate_expressions: Vec>>, + filter_expressions: Vec>>, + accumulators: Vec, + requirement: LexOrdering, +} impl AggregateStream { /// Create a new AggregateStream @@ -282,3 +282,20 @@ fn aggregate_batch( Ok(allocated) } + +/// returns a vector of ArrayRefs, where each entry corresponds to either the +/// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) +fn finalize_aggregation_groups( + aggregate_groups: &[AggregateGroup], + // TODO: Use Mapping indices + mode: &AggregateMode, +) -> Result> { + let elems = aggregate_groups + .iter() + .map(|aggregate_group| finalize_aggregation(&aggregate_group.accumulators, mode)) + .collect::>>()?; + // TODO: Add proper indices + // Convert Vec> to Vec. + let res = elems.into_iter().flatten().collect::>(); + Ok(res) +} From b848f4e19d436f0eddcf7ac750372a9091930413 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 10:42:57 +0300 Subject: [PATCH 26/49] Add proper indices handling --- .../physical-plan/src/aggregates/mod.rs | 58 ++++++++++++++++--- .../src/aggregates/no_grouping.rs | 22 ++++--- .../physical-plan/src/aggregates/row_hash.rs | 21 +++++-- .../sqllogictest/test_files/groupby.slt | 24 ++++++++ 4 files changed, 103 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ff2cadc2081e..cf095c6f4790 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -350,7 +350,7 @@ impl AggregateExec { )?; let eq_properties = input.equivalence_properties(); let mut aggregate_groups = - get_groups_indices(&mut aggr_expr, &group_by, &eq_properties, &mode); + get_aggregate_expr_groups(&mut aggr_expr, &group_by, &eq_properties, &mode); let schema = Arc::new(schema); @@ -863,7 +863,7 @@ fn get_req( req } -fn get_groups_indices( +fn get_aggregate_expr_groups( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, @@ -990,7 +990,7 @@ fn create_accumulators( fn finalize_aggregation( accumulators: &[AccumulatorItem], mode: &AggregateMode, -) -> Result> { +) -> Result>> { match mode { AggregateMode::Partial => { // build the vector of states @@ -1004,7 +1004,8 @@ fn finalize_aggregation( }) }) .collect::>>()?; - Ok(a.iter().flatten().cloned().collect::>()) + Ok(a) + // Ok(a.iter().flatten().cloned().collect::>()) } AggregateMode::Final | AggregateMode::FinalPartitioned @@ -1013,8 +1014,10 @@ fn finalize_aggregation( // merge the state to the final value accumulators .iter() - .map(|accumulator| accumulator.evaluate().and_then(|v| v.to_array())) - .collect::>>() + .map(|accumulator| { + accumulator.evaluate().and_then(|v| Ok(vec![v.to_array()?])) + }) + .collect::>>>() } } } @@ -1109,6 +1112,31 @@ pub(crate) fn evaluate_group_by( .collect()) } +fn reorder_aggregate_expr_results( + aggregate_group_results: Vec>>, + aggregate_group_indices: Vec>, +) -> Vec { + let n_aggregate = aggregate_group_indices + .iter() + .map(|group_indices| group_indices.len()) + .sum(); + + let mut result = vec![vec![]; n_aggregate]; + // Insert each aggregation result inside an aggregation group, to the proper places in the result + for (aggregate_group_result, group_indices) in aggregate_group_results + .into_iter() + .zip(aggregate_group_indices.iter()) + { + group_indices + .iter() + .zip(aggregate_group_result.into_iter()) + .for_each(|(&idx, aggr_state)| { + result[idx] = aggr_state; + }) + } + result.into_iter().flatten().collect() +} + #[cfg(test)] mod tests { use std::any::Any; @@ -2100,7 +2128,7 @@ mod tests { // let res = // get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; let group_by = PhysicalGroupBy::new_single(vec![]); - let res = get_groups_indices( + let res = get_aggregate_expr_groups( &mut aggr_exprs, &group_by, &eq_properties, @@ -2171,6 +2199,20 @@ mod tests { (vec![1], vec![(col_c, option_asc)]), ], ), + // ------- TEST CASE 6 ----------- + ( + // Ordering requirements + vec![ + vec![(col_a, option_asc)], + vec![(col_c, option_asc)], + vec![(col_a, option_desc)], + ], + // expected + vec![ + (vec![0, 2], vec![(col_a, option_asc)]), + (vec![1], vec![(col_c, option_asc)]), + ], + ), ]; for (ordering_reqs, expected) in test_cases { let mut aggr_exprs = ordering_reqs @@ -2192,7 +2234,7 @@ mod tests { // Empty equivalence Properties let eq_properties = EquivalenceProperties::new(test_schema.clone()); - let res = get_groups_indices( + let res = get_aggregate_expr_groups( &mut aggr_exprs, &group_by, &eq_properties, diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 5becd1389861..19419bf31c70 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -18,8 +18,8 @@ //! Aggregate without grouping columns use crate::aggregates::{ - aggregate_expressions, create_accumulators, finalize_aggregation, AccumulatorItem, - AggregateExprGroup, AggregateMode, + aggregate_expressions, create_accumulators, finalize_aggregation, + reorder_aggregate_expr_results, AccumulatorItem, AggregateExprGroup, AggregateMode, }; use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::{RecordBatchStream, SendableRecordBatchStream}; @@ -70,6 +70,7 @@ pub struct AggregateGroup { filter_expressions: Vec>>, accumulators: Vec, requirement: LexOrdering, + group_indices: Vec, } impl AggregateStream { @@ -116,6 +117,7 @@ impl AggregateStream { filter_expressions, accumulators, requirement: requirement.to_vec(), + group_indices: indices.to_vec(), }) }, ) @@ -287,15 +289,19 @@ fn aggregate_batch( /// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) fn finalize_aggregation_groups( aggregate_groups: &[AggregateGroup], - // TODO: Use Mapping indices mode: &AggregateMode, ) -> Result> { - let elems = aggregate_groups + let aggregate_group_results = aggregate_groups .iter() .map(|aggregate_group| finalize_aggregation(&aggregate_group.accumulators, mode)) .collect::>>()?; - // TODO: Add proper indices - // Convert Vec> to Vec. - let res = elems.into_iter().flatten().collect::>(); - Ok(res) + let aggregate_group_indices = aggregate_groups + .iter() + .map(|aggregate_group| aggregate_group.group_indices.to_vec()) + .collect::>(); + + Ok(reorder_aggregate_expr_results( + aggregate_group_results, + aggregate_group_indices, + )) } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 62c4cb38b769..a49d1571631f 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -25,7 +25,7 @@ use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ evaluate_group_by, evaluate_many, evaluate_optional, group_schema, - AggregateExprGroup, AggregateMode, PhysicalGroupBy, + reorder_aggregate_expr_results, AggregateExprGroup, AggregateMode, PhysicalGroupBy, }; use crate::common::IPCWriter; use crate::metrics::{BaselineMetrics, RecordOutput}; @@ -345,6 +345,7 @@ impl GroupedHashAggregateStream { merging_aggregate_arguments, filter_expressions, requirement: requirement.to_vec(), + group_indices: indices.to_vec(), }) }, ) @@ -356,6 +357,7 @@ impl GroupedHashAggregateStream { merging_aggregate_arguments: vec![], filter_expressions: vec![], requirement: vec![], + group_indices: vec![], }] } @@ -445,6 +447,7 @@ pub struct HashAggregateGroup { /// the filter expression is `x > 100`. filter_expressions: Vec>>, requirement: LexOrdering, + group_indices: Vec, } /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if @@ -734,26 +737,32 @@ impl GroupedHashAggregateStream { for acc in aggregate_group.accumulators.iter_mut() { match self.mode { AggregateMode::Partial => { - aggregate_group_output.extend(acc.state(emit_to)?) + aggregate_group_output.push(acc.state(emit_to)?) } _ if spilling => { // If spilling, output partial state because the spilled data will be // merged and re-evaluated later. - aggregate_group_output.extend(acc.state(emit_to)?) + aggregate_group_output.push(acc.state(emit_to)?) } AggregateMode::Final | AggregateMode::FinalPartitioned | AggregateMode::Single | AggregateMode::SinglePartitioned => { - aggregate_group_output.push(acc.evaluate(emit_to)?) + aggregate_group_output.push(vec![acc.evaluate(emit_to)?]) } } } Ok(aggregate_group_output) }) .collect::>>()?; - // TODO: Consider proper indices during merging. - let aggregate_outputs = outputs.into_iter().flatten().collect::>(); + let group_indices = self + .aggregate_groups + .iter() + .map(|aggregate_group| aggregate_group.group_indices.to_vec()) + .collect::>(); + let aggregate_outputs = reorder_aggregate_expr_results(outputs, group_indices); + // // TODO: Consider proper indices during merging. + // let aggregate_outputs = outputs.into_iter().flatten().collect::>(); output.extend(aggregate_outputs); // emit reduces the memory usage. Ignore Err from update_memory_reservation. Even if it is // over the target memory size after emission, we can emit again rather than returning Err. diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index aa1e0dcbafe0..38fc13a84872 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4286,3 +4286,27 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRA 0 1 3 0 [4, 4, 4, 3, 3, 3, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0, 0] 1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] + +# query IIII rowsort +# SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), FIRST_VALUE(d ORDER BY d DESC, c DESC) +# FROM multiple_ordered_table +# GROUP BY a, b +# ORDER BY a, b +# ---- + +query III rowsort +SELECT FIRST_VALUE(d ORDER BY c DESC), FIRST_VALUE(d ORDER BY d DESC, c DESC), FIRST_VALUE(d ORDER BY c ASC) + FROM multiple_ordered_table +---- +3 4 0 + +query IIIII rowsort +SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), FIRST_VALUE(d ORDER BY d DESC, c DESC), FIRST_VALUE(d ORDER BY c ASC) + FROM multiple_ordered_table + GROUP BY a, b + ORDER BY a, b +---- +0 0 4 4 0 +0 1 3 4 0 +1 2 4 4 0 +1 3 3 4 2 From 80a160c778919e27b557abbca55eb826ceaa5bbb Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 13:37:26 +0300 Subject: [PATCH 27/49] Minnor changes --- .../src/aggregates/no_grouping.rs | 138 ++++++++++-------- 1 file changed, 80 insertions(+), 58 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 19419bf31c70..bb42ec4a1dcc 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -42,6 +42,28 @@ use futures::stream::{Stream, StreamExt}; use super::AggregateExec; +/// A structure representing an aggregate group. +/// +/// The `AggregateGroup` struct is all aggregate expressions +/// where ordering requirement is satisfied by `requirement`. +/// This struct divides aggregate expressions according to their requirements. +/// Aggregate groups are constructed using `get_aggregate_expr_groups` function. +/// +/// # Fields +/// +/// - `aggregate_expressions`: A vector of vectors containing aggregate expressions. +/// - `filter_expressions`: A vector of optional filter expressions associated with each aggregate expression. +/// - `accumulators`: A vector of `AccumulatorItem` instances representing accumulators for the group. +/// - `requirement`: A `LexOrdering` instance specifying the lexical ordering requirement of the group. +/// - `group_indices`: A vector of indices indicating position of each aggregation in the original aggregate expression. +pub struct AggregateGroup { + aggregate_expressions: Vec>>, + filter_expressions: Vec>>, + accumulators: Vec, + requirement: LexOrdering, + group_indices: Vec, +} + /// stream struct for aggregation without grouping columns pub(crate) struct AggregateStream { stream: BoxStream<'static, Result>, @@ -65,14 +87,6 @@ struct AggregateStreamInner { finished: bool, } -pub struct AggregateGroup { - aggregate_expressions: Vec>>, - filter_expressions: Vec>>, - accumulators: Vec, - requirement: LexOrdering, - group_indices: Vec, -} - impl AggregateStream { /// Create a new AggregateStream pub fn new( @@ -110,7 +124,6 @@ impl AggregateStream { get_at_indices(&aggregate_expressions, indices)?; let filter_expressions = get_at_indices(&filter_expressions, indices)?; - // let accumulators = get_at_indices(&accumulators, &indices)?; let accumulators = create_accumulators(&aggr_exprs)?; Ok(AggregateGroup { aggregate_expressions, @@ -143,7 +156,7 @@ impl AggregateStream { let result = match this.input.next().await { Some(Ok(batch)) => { let timer = elapsed_compute.timer(); - let result = aggregate_batch( + let result = aggregate_batch_groups( &this.mode, batch, &mut this.aggregate_groups, @@ -215,17 +228,30 @@ impl RecordBatchStream for AggregateStream { } } -/// Perform group-by aggregation for the given [`RecordBatch`]. +/// Perform group-by aggregation for the given [`RecordBatch`] on all aggregate groups. +/// +/// If successful, this returns the additional number of bytes that were allocated during this process. +fn aggregate_batch_groups( + mode: &AggregateMode, + batch: RecordBatch, + aggregate_groups: &mut [AggregateGroup], +) -> Result { + let allocated = aggregate_groups + .iter_mut() + .map(|aggregate_group| aggregate_batch(mode, &batch, aggregate_group)) + .collect::>>()?; + Ok(allocated.into_iter().sum()) +} + +/// Perform group-by aggregation for the given [`RecordBatch`] on the aggregate group. /// /// If successful, this returns the additional number of bytes that were allocated during this process. /// /// TODO: Make this a member function fn aggregate_batch( mode: &AggregateMode, - batch: RecordBatch, - aggregate_groups: &mut [AggregateGroup], - // expressions: &[Vec>], - // filters: &[Option>], + batch: &RecordBatch, + aggregate_group: &mut AggregateGroup, ) -> Result { let mut allocated = 0usize; @@ -234,52 +260,48 @@ fn aggregate_batch( // 1.3 evaluate expressions // 1.4 update / merge accumulators with the expressions' values - aggregate_groups + let accumulators = &mut aggregate_group.accumulators; + let expressions = &mut aggregate_group.aggregate_expressions; + let filters = &mut aggregate_group.filter_expressions; + let requirement = &aggregate_group.requirement; + let batch = if requirement.is_empty() { + batch.clone() + } else { + sort_batch(batch, requirement, None)? + }; + // 1.1 + accumulators .iter_mut() - .try_for_each(|aggregate_group| { - let accumulators = &mut aggregate_group.accumulators; - let expressions = &mut aggregate_group.aggregate_expressions; - let filters = &mut aggregate_group.filter_expressions; - let requirement = &aggregate_group.requirement; - let batch = if requirement.is_empty() { - batch.clone() - } else { - sort_batch(&batch, requirement, None)? + .zip(expressions) + .zip(filters) + .try_for_each(|((accum, expr), filter)| { + // 1.2 + let batch = match filter { + Some(filter) => Cow::Owned(batch_filter(&batch, filter)?), + None => Cow::Borrowed(&batch), }; - // 1.1 - accumulators - .iter_mut() - .zip(expressions) - .zip(filters) - .try_for_each(|((accum, expr), filter)| { - // 1.2 - let batch = match filter { - Some(filter) => Cow::Owned(batch_filter(&batch, filter)?), - None => Cow::Borrowed(&batch), - }; - // 1.3 - let values = &expr - .iter() - .map(|e| { - e.evaluate(&batch) - .and_then(|v| v.into_array(batch.num_rows())) - }) - .collect::>>()?; - - // 1.4 - let size_pre = accum.size(); - let res = match mode { - AggregateMode::Partial - | AggregateMode::Single - | AggregateMode::SinglePartitioned => accum.update_batch(values), - AggregateMode::Final | AggregateMode::FinalPartitioned => { - accum.merge_batch(values) - } - }; - let size_post = accum.size(); - allocated += size_post.saturating_sub(size_pre); - res + // 1.3 + let values = &expr + .iter() + .map(|e| { + e.evaluate(&batch) + .and_then(|v| v.into_array(batch.num_rows())) }) + .collect::>>()?; + + // 1.4 + let size_pre = accum.size(); + let res = match mode { + AggregateMode::Partial + | AggregateMode::Single + | AggregateMode::SinglePartitioned => accum.update_batch(values), + AggregateMode::Final | AggregateMode::FinalPartitioned => { + accum.merge_batch(values) + } + }; + let size_post = accum.size(); + allocated += size_post.saturating_sub(size_pre); + res })?; Ok(allocated) From 89e0c2b1a021ce8bb9d3f945d82e33549131022c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 13:42:03 +0300 Subject: [PATCH 28/49] Minor changes --- .../src/aggregates/no_grouping.rs | 1 + .../physical-plan/src/aggregates/row_hash.rs | 40 +------------------ 2 files changed, 3 insertions(+), 38 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index bb42ec4a1dcc..2c40f6741f92 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -99,6 +99,7 @@ impl AggregateStream { let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); let input = agg.input.execute(partition, Arc::clone(&context))?; + let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; let filter_expressions = match agg.mode { AggregateMode::Partial diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index a49d1571631f..4c41168798f5 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -36,7 +36,6 @@ use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; -use arrow::util::pretty::print_batches; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use arrow_schema::SortOptions; use datafusion_common::{DataFusionError, Result}; @@ -56,8 +55,6 @@ use futures::ready; use futures::stream::{Stream, StreamExt}; use log::debug; -const PRINT_ON: bool = false; - #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) pub(crate) enum ExecutionState { @@ -210,30 +207,9 @@ pub(crate) struct GroupedHashAggregateStream { input: SendableRecordBatchStream, mode: AggregateMode, + /// Stores aggregate groups that have different ordering requirements + /// Aggregate groups are calculated using `get_aggregate_expr_groups` function. aggregate_groups: Vec, - // /// Accumulators, one for each `AggregateExpr` in the query - // /// - // /// For example, if the query has aggregates, `SUM(x)`, - // /// `COUNT(y)`, there will be two accumulators, each one - // /// specialized for that particular aggregate and its input types - // accumulators: Vec>, - // - // /// Arguments to pass to each accumulator. - // /// - // /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` - // /// - // /// The argument to each accumulator is itself a `Vec` because - // /// some aggregates such as `CORR` can accept more than one - // /// argument. - // aggregate_arguments: Vec>>, - // - // /// Optional filter expression to evaluate, one for each for - // /// accumulator. If present, only those rows for which the filter - // /// evaluate to true should be included in the aggregate results. - // /// - // /// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`, - // /// the filter expression is `x > 100`. - // filter_expressions: Vec>>, /// GROUP BY expressions group_by: PhysicalGroupBy, @@ -586,23 +562,11 @@ impl GroupedHashAggregateStream { /// Perform group-by aggregation for the given [`RecordBatch`]. fn group_aggregate_batch(&mut self, batch: RecordBatch) -> Result<()> { for aggregate_group in &mut self.aggregate_groups { - if PRINT_ON { - println!( - "aggregate_group.requirement: {:?}", - aggregate_group.requirement - ); - println!("before sort"); - print_batches(&[batch.clone()])?; - } let batch = if aggregate_group.requirement.is_empty() { batch.clone() } else { sort_batch(&batch, &aggregate_group.requirement, None)? }; - if PRINT_ON { - print_batches(&[batch.clone()])?; - println!("after sort"); - } // Evaluate the grouping expressions let group_by_values = if self.spill_state.is_stream_merging { evaluate_group_by(&self.spill_state.merging_group_by, &batch)? From 299e89cb02a578ce184fc1b1f2bca7baf095a727 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 14:13:20 +0300 Subject: [PATCH 29/49] Minor changes --- .../physical-plan/src/aggregates/mod.rs | 71 +++++++++++++------ .../physical-plan/src/aggregates/row_hash.rs | 67 ++++++++--------- 2 files changed, 83 insertions(+), 55 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index cf095c6f4790..51044e8fb190 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -37,7 +37,7 @@ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{exec_datafusion_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ @@ -268,6 +268,9 @@ impl From for SendableRecordBatchStream { } } +/// A structure representing a group of aggregate expressions where each group has different +/// ordering requirements. Aggregate groups are calculated using `get_aggregate_expr_groups` function. +/// Indices refers to the position of each aggregate expressions among all aggregate expressions (prior to grouping). #[derive(Clone, Debug, PartialEq)] pub struct AggregateExprGroup { /// Aggregate expressions indices @@ -289,6 +292,7 @@ pub struct AggregateExec { filter_expr: Vec>>, /// (ORDER BY clause) expression for each aggregate expression order_by_expr: Vec>, + /// Stores aggregate groups where each group has different ordering requirement. aggregate_groups: Vec, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause limit: Option, @@ -350,7 +354,7 @@ impl AggregateExec { )?; let eq_properties = input.equivalence_properties(); let mut aggregate_groups = - get_aggregate_expr_groups(&mut aggr_expr, &group_by, &eq_properties, &mode); + get_aggregate_expr_groups(&mut aggr_expr, &group_by, &eq_properties, &mode)?; let schema = Arc::new(schema); @@ -837,7 +841,18 @@ fn group_schema(schema: &Schema, group_count: usize) -> SchemaRef { Arc::new(Schema::new(group_fields)) } -fn get_req( +/// 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, @@ -854,6 +869,7 @@ fn get_req( // Hence, we do not need to use the ordering requirement in such // modes as long as partial results are generated with the // correct ordering. + // TODO: Remove all orderings that occur in the group by. if !is_order_sensitive(aggr_expr) || group_by_contains_all_requirements(group_by, &req) || !agg_mode.is_first_stage() @@ -863,12 +879,25 @@ fn get_req( req } +/// Groups aggregate expressions based on their ordering requirements. +/// +/// # Parameters +/// +/// - `aggr_exprs`: A mutable slice of `Arc` representing the aggregate expressions to be grouped. +/// - `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 vector of `AggregateExprGroup` instances, each containing indices and ordering requirements for a group of +/// related aggregate expressions. fn get_aggregate_expr_groups( aggr_exprs: &mut [Arc], group_by: &PhysicalGroupBy, eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, -) -> Vec { +) -> Result> { let mut used_indices: HashSet = HashSet::new(); let mut groups = vec![]; while used_indices.len() != aggr_exprs.len() { @@ -879,13 +908,14 @@ fn get_aggregate_expr_groups( // Skip this group, it is already inserted. continue; } - let aggr_req = get_req(aggr_expr, group_by, agg_mode); + let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); if let Some((group_indices, req)) = &mut group { if let Some(finer) = eq_properties.get_finer_ordering(req, &aggr_req) { *req = finer; group_indices.push(idx); } else if let Some(reverse) = aggr_expr.reverse_expr() { - let reverse_req = get_req(&reverse, group_by, agg_mode); + let reverse_req = + get_aggregate_expr_req(&reverse, group_by, agg_mode); if let Some(finer) = eq_properties.get_finer_ordering(req, &reverse_req) { @@ -898,18 +928,17 @@ fn get_aggregate_expr_groups( group = Some((vec![idx], aggr_req)); } } - if let Some((indices, requirement)) = group { - used_indices.extend(indices.iter()); - groups.push(AggregateExprGroup { - indices, - requirement, - }); - } else { - unreachable!(); - } + // We cannot received None here. + let (indices, requirement) = + group.ok_or_else(|| exec_datafusion_err!("Cannot Receive empty group"))?; + used_indices.extend(indices.iter()); + groups.push(AggregateExprGroup { + indices, + requirement, + }); } - groups + Ok(groups) } /// returns physical expressions for arguments to evaluate against a batch @@ -994,7 +1023,7 @@ fn finalize_aggregation( match mode { AggregateMode::Partial => { // build the vector of states - let a = accumulators + accumulators .iter() .map(|accumulator| { accumulator.state().and_then(|e| { @@ -1003,9 +1032,7 @@ fn finalize_aggregation( .collect::>>() }) }) - .collect::>>()?; - Ok(a) - // Ok(a.iter().flatten().cloned().collect::>()) + .collect::>>() } AggregateMode::Final | AggregateMode::FinalPartitioned @@ -2133,7 +2160,7 @@ mod tests { &group_by, &eq_properties, &AggregateMode::Partial, - ); + )?; let res = res[0].requirement.clone(); assert_eq!(res, common_requirement); Ok(()) @@ -2239,7 +2266,7 @@ mod tests { &group_by, &eq_properties, &AggregateMode::Partial, - ); + )?; let expected = expected .into_iter() diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 4c41168798f5..cec51c11b1c6 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -87,6 +87,40 @@ struct SpillState { merging_group_by: PhysicalGroupBy, } +pub struct HashAggregateGroup { + /// Accumulators, one for each `AggregateExpr` in the query + /// + /// For example, if the query has aggregates, `SUM(x)`, + /// `COUNT(y)`, there will be two accumulators, each one + /// specialized for that particular aggregate and its input types + accumulators: Vec>, + + /// Arguments to pass to each accumulator. + /// + /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` + /// + /// The argument to each accumulator is itself a `Vec` because + /// some aggregates such as `CORR` can accept more than one + /// argument. + aggregate_arguments: Vec>>, + + /// aggregate_arguments at the input of the merging (may have additional fields) + merging_aggregate_arguments: Vec>>, + + /// Optional filter expression to evaluate, one for each for + /// accumulator. If present, only those rows for which the filter + /// evaluate to true should be included in the aggregate results. + /// + /// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`, + /// the filter expression is `x > 100`. + filter_expressions: Vec>>, + /// Ordering Requirement for the aggregate group + requirement: LexOrdering, + /// Indices stores the position of the each aggregate expression + /// among all aggregate expressions (prior to grouping). + group_indices: Vec, +} + /// HashTable based Grouping Aggregator /// /// # Design Goals @@ -395,37 +429,6 @@ impl GroupedHashAggregateStream { } } -pub struct HashAggregateGroup { - /// Accumulators, one for each `AggregateExpr` in the query - /// - /// For example, if the query has aggregates, `SUM(x)`, - /// `COUNT(y)`, there will be two accumulators, each one - /// specialized for that particular aggregate and its input types - accumulators: Vec>, - - /// Arguments to pass to each accumulator. - /// - /// The arguments in `accumulator[i]` is passed `aggregate_arguments[i]` - /// - /// The argument to each accumulator is itself a `Vec` because - /// some aggregates such as `CORR` can accept more than one - /// argument. - aggregate_arguments: Vec>>, - - /// aggregate_arguments at the input of the merging (may have additional fields) - merging_aggregate_arguments: Vec>>, - - /// Optional filter expression to evaluate, one for each for - /// accumulator. If present, only those rows for which the filter - /// evaluate to true should be included in the aggregate results. - /// - /// For example, for an aggregate like `SUM(x) FILTER (WHERE x >= 100)`, - /// the filter expression is `x > 100`. - filter_expressions: Vec>>, - requirement: LexOrdering, - group_indices: Vec, -} - /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. @@ -725,8 +728,6 @@ impl GroupedHashAggregateStream { .map(|aggregate_group| aggregate_group.group_indices.to_vec()) .collect::>(); let aggregate_outputs = reorder_aggregate_expr_results(outputs, group_indices); - // // TODO: Consider proper indices during merging. - // let aggregate_outputs = outputs.into_iter().flatten().collect::>(); output.extend(aggregate_outputs); // emit reduces the memory usage. Ignore Err from update_memory_reservation. Even if it is // over the target memory size after emission, we can emit again rather than returning Err. From a2723e5c1467fada1b4f822db6ebadc4a26a9de5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 14:16:32 +0300 Subject: [PATCH 30/49] Minor changes --- datafusion/physical-plan/src/aggregates/mod.rs | 2 ++ datafusion/sqllogictest/test_files/groupby.slt | 16 ++++++++-------- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 51044e8fb190..a3777a261ce4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1139,6 +1139,8 @@ pub(crate) fn evaluate_group_by( .collect()) } +/// Inserts each aggregate expression result inside aggregate groups +/// to correct indices in the output. fn reorder_aggregate_expr_results( aggregate_group_results: Vec>>, aggregate_group_indices: Vec>, diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 38fc13a84872..3da04af22692 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2261,7 +2261,7 @@ CREATE TABLE sales_global (zip_code INT, (1, 'TUR', 2, '2022-01-01 11:30:00'::timestamp, 'TRY', 75.0), (1, 'FRA', 3, '2022-01-02 12:00:00'::timestamp, 'EUR', 200.0), (1, 'TUR', 4, '2022-01-03 10:00:00'::timestamp, 'TRY', 100.0), - (0, 'GRC', 4, '2022-01-03 11:00:00'::timestamp, 'EUR', 80.0) + (0, 'GRC', 4, '2022-01-03 10:00:00'::timestamp, 'EUR', 80.0) # create a new table named exchange rates statement ok @@ -2697,16 +2697,16 @@ ProjectionExec: expr=[country@0 as country, FIRST_VALUE(sales_global.amount) ORD ----SortExec: expr=[ts@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] -query TRRR +query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, LAST_VALUE(amount ORDER BY ts DESC) as lv1, SUM(amount ORDER BY ts DESC) as sum1 FROM sales_global GROUP BY country ---- +FRA 200 50 250 GRC 80 30 110 TUR 100 75 175 -FRA 200 50 250 query TT EXPLAIN SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2738,7 +2738,7 @@ SortExec: expr=[sn@2 ASC NULLS LAST] --------------MemoryExec: partitions=1, partition_sizes=[1] --------------MemoryExec: partitions=1, partition_sizes=[1] -query ITIPTR +query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate FROM sales_global AS s JOIN sales_global AS e @@ -2748,10 +2748,10 @@ GROUP BY s.sn, s.zip_code, s.country, s.ts, s.currency ORDER BY s.sn ---- 0 GRC 0 2022-01-01T06:00:00 EUR 30 +0 GRC 4 2022-01-03T10:00:00 EUR 80 1 FRA 1 2022-01-01T08:00:00 EUR 50 -1 TUR 2 2022-01-01T11:30:00 TRY 75 1 FRA 3 2022-01-02T12:00:00 EUR 200 -0 GRC 4 2022-01-03T11:00:00 EUR 80 +1 TUR 2 2022-01-01T11:30:00 TRY 75 1 TUR 4 2022-01-03T10:00:00 TRY 100 # Run order-sensitive aggregators in multiple partitions @@ -2937,7 +2937,7 @@ query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 FROM sales_global ---- -[80.0, 100.0, 200.0, 75.0, 50.0, 30.0] +[100.0, 80.0, 200.0, 75.0, 50.0, 30.0] # ARRAY_AGG should work in multiple partitions query TT @@ -3081,7 +3081,7 @@ ORDER BY s.sn 1 FRA 1 2022-01-01T08:00:00 EUR 50 1 TUR 2 2022-01-01T11:30:00 TRY 75 1 FRA 3 2022-01-02T12:00:00 EUR 200 -0 GRC 4 2022-01-03T11:00:00 EUR 80 +0 GRC 4 2022-01-03T10:00:00 EUR 80 1 TUR 4 2022-01-03T10:00:00 TRY 100 # create a table for testing From 373dd88e3b24484b69eaaef88305ee45d044fefe Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 6 Dec 2023 17:20:58 +0300 Subject: [PATCH 31/49] Remove requirement originating from order sensitive aggregators --- .../physical-plan/src/aggregates/mod.rs | 47 ++----- .../sqllogictest/test_files/distinct_on.slt | 16 +-- .../sqllogictest/test_files/groupby.slt | 129 +++++++----------- datafusion/sqllogictest/test_files/joins.slt | 4 +- 4 files changed, 72 insertions(+), 124 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a3777a261ce4..8e8b69d9d755 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -316,23 +316,6 @@ pub struct AggregateExec { output_ordering: Option, } -/// Check whether group by expression contains all of the expression inside `requirement` -// As an example Group By (c,b,a) contains all of the expressions in the `requirement`: (a ASC, b DESC) -fn group_by_contains_all_requirements( - group_by: &PhysicalGroupBy, - requirement: &LexOrdering, -) -> bool { - let physical_exprs = group_by.input_exprs(); - // When we have multiple groups (grouping set) - // since group by may be calculated on the subset of the group_by.expr() - // it is not guaranteed to have all of the requirements among group by expressions. - // Hence do the analysis: whether group by contains all requirements in the single group case. - group_by.is_single() - && requirement - .iter() - .all(|req| physical_exprs_contains(&physical_exprs, &req.expr)) -} - impl AggregateExec { /// Create a new hash aggregate execution plan pub fn try_new( @@ -353,21 +336,11 @@ impl AggregateExec { mode, )?; let eq_properties = input.equivalence_properties(); - let mut aggregate_groups = + let aggregate_groups = get_aggregate_expr_groups(&mut aggr_expr, &group_by, &eq_properties, &mode)?; let schema = Arc::new(schema); - let mut ordering_req = vec![]; - if let Some(max_group) = aggregate_groups - .iter_mut() - .max_by(|lhs, rhs| lhs.indices.len().cmp(&rhs.indices.len())) - { - ordering_req = max_group.requirement.to_vec(); - // No longer require this, this requirement will be satisfied by outside mechanism. - max_group.requirement.clear(); - } - // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of the GROUP BY expressions, @@ -381,9 +354,6 @@ impl AggregateExec { options: None, }) .collect::>(); - // Postfix ordering requirement of the aggregation to the requirement. - let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); - new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); let partition_search_mode = @@ -869,11 +839,15 @@ fn get_aggregate_expr_req( // Hence, we do not need to use the ordering requirement in such // modes as long as partial results are generated with the // correct ordering. - // TODO: Remove all orderings that occur in the group by. - if !is_order_sensitive(aggr_expr) - || group_by_contains_all_requirements(group_by, &req) - || !agg_mode.is_first_stage() - { + if group_by.is_single() { + // Remove all orderings that occur in the group by. These requirements will be satisfied definitely + // (Per group each group by expression will have distinct values. Hence all requirements are satisfied). + let physical_exprs = group_by.input_exprs(); + req.retain(|sort_expr| { + !physical_exprs_contains(&physical_exprs, &sort_expr.expr) + }); + } + if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { req.clear(); } req @@ -925,6 +899,7 @@ fn get_aggregate_expr_groups( } } } else { + // Initialize group with current aggregate expression group = Some((vec![idx], aggr_req)); } } diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index 8a36b49b98c6..e33b3efe3a52 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -38,17 +38,16 @@ LOCATION '../../testing/data/csv/aggregate_test_100.csv' # Basic example: distinct on the first column project the second one, and # order by the third query TI -SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3; +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3, c9; ---- -a 5 +a 4 b 4 c 2 d 1 e 3 -# Basic example + reverse order of the selected column query TI -SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3 DESC; +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1, c3 DESC, c9; ---- a 1 b 5 @@ -58,7 +57,7 @@ e 1 # Basic example + reverse order of the ON column query TI -SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1 DESC, c3; +SELECT DISTINCT ON (c1) c1, c2 FROM aggregate_test_100 ORDER BY c1 DESC, c3, c9; ---- e 3 d 1 @@ -100,10 +99,9 @@ ProjectionExec: expr=[FIRST_VALUE(aggregate_test_100.c3) ORDER BY [aggregate_tes ------AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)], ordering_mode=Sorted ---------------SortExec: expr=[c1@0 ASC NULLS LAST,c3@2 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true +------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[FIRST_VALUE(aggregate_test_100.c3), FIRST_VALUE(aggregate_test_100.c2)] +--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # ON expressions are not a sub-set of the ORDER BY expressions query error SELECT DISTINCT ON expressions must match initial ORDER BY expressions diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 3da04af22692..e48373643780 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,16 +2019,15 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) ---------------SortExec: expr=[col0@3 ASC NULLS LAST] -----------------CoalesceBatchesExec: target_batch_size=8192 -------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] ---------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 -------------------------MemoryExec: partitions=1, partition_sizes=[3] +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] +--------------CoalesceBatchesExec: target_batch_size=8192 +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[3] +------------------CoalesceBatchesExec: target_batch_size=8192 +--------------------RepartitionExec: partitioning=Hash([col0@0], 4), input_partitions=1 +----------------------MemoryExec: partitions=1, partition_sizes=[3] # Columns in the table are a,b,c,d. Source is CsvExec which is ordered by # a,b,c column. Column a has cardinality 2, column b has cardinality 4. @@ -2296,8 +2295,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@1 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query T? @@ -2326,8 +2324,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2357,8 +2354,7 @@ physical_plan SortExec: expr=[country@0 ASC NULLS LAST] --ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, ARRAY_AGG(s.amount) ORDER BY [s.amount ASC NULLS LAST]@2 as amounts2, ARRAY_AGG(s.amount) ORDER BY [s.sn ASC NULLS LAST]@3 as amounts3] ----AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), ARRAY_AGG(s.amount), ARRAY_AGG(s.amount)] -------SortExec: expr=[sn@1 ASC NULLS LAST] ---------MemoryExec: partitions=1, partition_sizes=[1] +------MemoryExec: partitions=1, partition_sizes=[1] query T??? rowsort SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, @@ -2393,7 +2389,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2429,7 +2425,7 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N physical_plan ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] --AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) -----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] +----SortExec: expr=[country@1 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] query TI?R rowsort @@ -2500,7 +2496,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted -----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] +----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] query T?R rowsort @@ -2532,8 +2528,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales 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)] -----SortExec: expr=[amount@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query T?RR rowsort SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -2563,8 +2558,7 @@ Projection: sales_global.country, ARRAY_AGG(sales_global.amount) ORDER BY [sales 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)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS amounts, @@ -2595,8 +2589,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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)] -----SortExec: expr=[amount@1 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TRR? SELECT country, FIRST_VALUE(amount ORDER BY amount ASC) AS fv1, @@ -2625,8 +2618,7 @@ Projection: sales_global.country, SUM(sales_global.amount) ORDER BY [sales_globa physical_plan ProjectionExec: expr=[country@0 as country, SUM(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]@1 as sum1, ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]@2 as amounts] --AggregateExec: mode=Single, gby=[country@0 as country], aggr=[SUM(sales_global.amount), ARRAY_AGG(sales_global.amount)] -----SortExec: expr=[amount@2 ASC NULLS LAST] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TR? SELECT country, SUM(amount ORDER BY ts DESC) AS sum1, @@ -2660,8 +2652,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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)] -----SortExec: expr=[ts@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2694,8 +2685,7 @@ Projection: sales_global.country, FIRST_VALUE(sales_global.amount) ORDER BY [sal 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)] -----SortExec: expr=[ts@1 DESC] -------MemoryExec: partitions=1, partition_sizes=[1] +----MemoryExec: partitions=1, partition_sizes=[1] query TRRR rowsort SELECT country, FIRST_VALUE(amount ORDER BY ts DESC) as fv1, @@ -2731,12 +2721,11 @@ physical_plan SortExec: expr=[sn@2 ASC NULLS LAST] --ProjectionExec: expr=[zip_code@1 as zip_code, country@2 as country, sn@0 as sn, ts@3 as ts, currency@4 as currency, LAST_VALUE(e.amount) ORDER BY [e.sn ASC NULLS LAST]@5 as last_rate] ----AggregateExec: mode=Single, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency], aggr=[LAST_VALUE(e.amount)] -------SortExec: expr=[sn@5 ASC NULLS LAST] ---------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] -----------CoalesceBatchesExec: target_batch_size=8192 -------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 ---------------MemoryExec: partitions=1, partition_sizes=[1] ---------------MemoryExec: partitions=1, partition_sizes=[1] +------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, sn@5 as sn, amount@8 as amount] +--------CoalesceBatchesExec: target_batch_size=8192 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(currency@4, currency@2)], filter=ts@0 >= ts@1 +------------MemoryExec: partitions=1, partition_sizes=[1] +------------MemoryExec: partitions=1, partition_sizes=[1] query ITIPTR rowsort SELECT s.zip_code, s.country, s.sn, s.ts, s.currency, LAST_VALUE(e.amount ORDER BY e.sn) AS last_rate @@ -2781,8 +2770,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] -----------------SortExec: expr=[ts@1 ASC NULLS LAST] -------------------MemoryExec: partitions=1, partition_sizes=[1] +----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2818,8 +2806,7 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 --------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] -----------------SortExec: expr=[ts@1 ASC NULLS LAST] -------------------MemoryExec: partitions=1, partition_sizes=[1] +----------------MemoryExec: partitions=1, partition_sizes=[1] query TRR SELECT country, FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2853,9 +2840,8 @@ ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), LAST_VALUE(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2880,9 +2866,8 @@ ProjectionExec: expr=[FIRST_VALUE(sales_global.amount) ORDER BY [sales_global.ts --AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(sales_global.amount), FIRST_VALUE(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query RR SELECT FIRST_VALUE(amount ORDER BY ts ASC) AS fv1, @@ -2905,9 +2890,8 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts A --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[ts@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts ASC) AS array_agg1 @@ -2929,9 +2913,8 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.ts D --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[ts@0 DESC] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY ts DESC) AS array_agg1 @@ -2953,9 +2936,8 @@ ProjectionExec: expr=[ARRAY_AGG(sales_global.amount) ORDER BY [sales_global.amou --AggregateExec: mode=Final, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ----CoalescePartitionsExec ------AggregateExec: mode=Partial, gby=[], aggr=[ARRAY_AGG(sales_global.amount)] ---------SortExec: expr=[amount@0 ASC NULLS LAST] -----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------MemoryExec: partitions=1, partition_sizes=[1] +--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------MemoryExec: partitions=1, partition_sizes=[1] query ? SELECT ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -2983,9 +2965,8 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount)] ---------------SortExec: expr=[amount@1 ASC NULLS LAST] -----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -------------------MemoryExec: partitions=1, partition_sizes=[1] +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] query T? SELECT country, ARRAY_AGG(amount ORDER BY amount ASC) AS array_agg1 @@ -3019,9 +3000,8 @@ SortPreservingMergeExec: [country@0 ASC NULLS LAST] --------CoalesceBatchesExec: target_batch_size=4 ----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 ------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[ARRAY_AGG(sales_global.amount), 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] +--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +----------------MemoryExec: partitions=1, partition_sizes=[1] query T?RR SELECT country, ARRAY_AGG(amount ORDER BY amount DESC) AS amounts, @@ -3650,9 +3630,8 @@ ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_o ----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)] -----------SortExec: expr=[c@1 DESC] -------------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 +----------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 query II rowsort SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, @@ -4091,9 +4070,8 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -----------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC] -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -4139,9 +4117,8 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ----CoalesceBatchesExec: target_batch_size=2 ------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -----------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 DESC] -------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ---------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -4195,9 +4172,8 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=20 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 ----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -------------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,d@3 DESC] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; @@ -4244,9 +4220,8 @@ SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] ------CoalesceBatchesExec: target_batch_size=20 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 ----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted -------------SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST,c@2 DESC] ---------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 -----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true +------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok set datafusion.execution.target_partitions = 1; diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 0fea8da5a342..777b1f6b2706 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3456,7 +3456,7 @@ SortPreservingMergeExec: [a@0 ASC] ------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] --------CoalesceBatchesExec: target_batch_size=2 ----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 -------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] --------------CoalesceBatchesExec: target_batch_size=2 ----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] ------------------CoalesceBatchesExec: target_batch_size=2 @@ -3464,7 +3464,7 @@ SortPreservingMergeExec: [a@0 ASC] ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true ------------------CoalesceBatchesExec: target_batch_size=2 ---------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST +--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true From f375bc4670a6fb7099199442e847c9d72300230f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 7 Dec 2023 10:11:54 +0300 Subject: [PATCH 32/49] Minor changes --- .../physical-plan/src/aggregates/mod.rs | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7dafe8b9b43e..4ef8e134a9f9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -882,6 +882,32 @@ fn get_aggregate_expr_req( 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 existing requirement and requirement of the +/// aggregator 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) +} + /// Groups aggregate expressions based on their ordering requirements. /// /// # Parameters @@ -911,16 +937,15 @@ fn get_aggregate_expr_groups( // Skip this group, it is already inserted. continue; } - let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); if let Some((group_indices, req)) = &mut group { - if let Some(finer) = eq_properties.get_finer_ordering(req, &aggr_req) { + if let Some(finer) = + finer_ordering(req, aggr_expr, group_by, eq_properties, agg_mode) + { *req = finer; group_indices.push(idx); } else if let Some(reverse) = aggr_expr.reverse_expr() { - let reverse_req = - get_aggregate_expr_req(&reverse, group_by, agg_mode); if let Some(finer) = - eq_properties.get_finer_ordering(req, &reverse_req) + finer_ordering(req, &reverse, group_by, eq_properties, agg_mode) { *aggr_expr = reverse; *req = finer; @@ -928,6 +953,7 @@ fn get_aggregate_expr_groups( } } } else { + let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); // Initialize group with current aggregate expression group = Some((vec![idx], aggr_req)); } From 96ac322349b9115aa7ce5f6167a61b7f81b12515 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 7 Dec 2023 10:24:46 +0300 Subject: [PATCH 33/49] Minor changes --- .../physical-plan/src/aggregates/mod.rs | 32 ++++++++++--------- .../physical-plan/src/aggregates/row_hash.rs | 29 ++++++++++++----- 2 files changed, 38 insertions(+), 23 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4ef8e134a9f9..76cf2450b31d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -937,26 +937,28 @@ fn get_aggregate_expr_groups( // Skip this group, it is already inserted. continue; } - if let Some((group_indices, req)) = &mut group { + let (mut group_indices, mut req) = if let Some((group_indices, req)) = group { + (group_indices, req) + } else { + // Initialize an empty group with empty requirement + (vec![], vec![]) + }; + if let Some(finer) = + finer_ordering(&req, aggr_expr, group_by, eq_properties, agg_mode) + { + req = finer; + group_indices.push(idx); + } else if let Some(reverse) = aggr_expr.reverse_expr() { if let Some(finer) = - finer_ordering(req, aggr_expr, group_by, eq_properties, agg_mode) + finer_ordering(&req, &reverse, group_by, eq_properties, agg_mode) { - *req = finer; + *aggr_expr = reverse; + req = finer; group_indices.push(idx); - } else if let Some(reverse) = aggr_expr.reverse_expr() { - if let Some(finer) = - finer_ordering(req, &reverse, group_by, eq_properties, agg_mode) - { - *aggr_expr = reverse; - *req = finer; - group_indices.push(idx); - } } - } else { - let aggr_req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); - // Initialize group with current aggregate expression - group = Some((vec![idx], aggr_req)); } + // Update group with new entries + group = Some((group_indices, req)); } // We cannot received None here. let (indices, requirement) = diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index a6063169ad48..0ba1fa83316e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -121,6 +121,20 @@ pub struct HashAggregateGroup { group_indices: Vec, } +impl HashAggregateGroup { + /// Initialize an empty group. + fn empty() -> Self { + HashAggregateGroup { + accumulators: vec![], + aggregate_arguments: vec![], + merging_aggregate_arguments: vec![], + filter_expressions: vec![], + requirement: vec![], + group_indices: vec![], + } + } +} + /// HashTable based Grouping Aggregator /// /// # Design Goals @@ -361,14 +375,13 @@ impl GroupedHashAggregateStream { ) .collect::>>()?; if aggregate_groups.is_empty() { - aggregate_groups = vec![HashAggregateGroup { - accumulators: vec![], - aggregate_arguments: vec![], - merging_aggregate_arguments: vec![], - filter_expressions: vec![], - requirement: vec![], - group_indices: vec![], - }] + // Make sure there is at least one empty group. + // This situations can arise in queries in following form + // SELECT a, b + // FROM table + // GROUP BY a,b + // where there is no aggregate expression. + aggregate_groups = vec![HashAggregateGroup::empty()] } // we need to use original schema so RowConverter in group_values below From 163b05673f4bf25336d03a15ef7f998adeb5d68f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 7 Dec 2023 10:33:15 +0300 Subject: [PATCH 34/49] Minor changes --- .../sqllogictest/test_files/groupby.slt | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index e48373643780..8f087964df28 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4148,13 +4148,10 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRA 1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] -# Test same queries where there is sort inserted +# Test same queries where there is sort inserted (source ordering is not preserved) statement ok set datafusion.optimizer.prefer_existing_sort = false; -statement ok -set datafusion.execution.batch_size = 20; - query TT EXPLAIN SELECT a, b, ARRAY_AGG(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table @@ -4169,7 +4166,7 @@ physical_plan SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] --AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=20 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 ----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -4217,7 +4214,7 @@ physical_plan SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] --AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----SortExec: expr=[a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] -------CoalesceBatchesExec: target_batch_size=20 +------CoalesceBatchesExec: target_batch_size=2 --------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8 ----------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 @@ -4237,6 +4234,7 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRA 1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] +# Test conflicting ordering requirement in single partition query IIII rowsort SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) FROM multiple_ordered_table @@ -4251,6 +4249,7 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY d DESC) statement ok set datafusion.execution.target_partitions = 8; +# Test conflicting ordering requirement in multiple partitions query IIII? SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRAY_AGG(d ORDER BY d DESC) FROM multiple_ordered_table @@ -4262,19 +4261,16 @@ SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), LAST_VALUE(d ORDER BY c DESC), ARRA 1 2 4 0 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 0, 0, 0] 1 3 3 2 [4, 4, 4, 4, 4, 4, 3, 3, 2, 2, 2, 2, 2, 2, 2, 1, 1, 1, 1, 1, 0, 0, 0, 0, 0] -# query IIII rowsort -# SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), FIRST_VALUE(d ORDER BY d DESC, c DESC) -# FROM multiple_ordered_table -# GROUP BY a, b -# ORDER BY a, b -# ---- - +# Test a query with conflicting ordering requirements (where aggregate groups are not consecutive. +# For query below aggregate groups are at indices [0, 2] and [1]). Please note that there is no group by query III rowsort SELECT FIRST_VALUE(d ORDER BY c DESC), FIRST_VALUE(d ORDER BY d DESC, c DESC), FIRST_VALUE(d ORDER BY c ASC) FROM multiple_ordered_table ---- 3 4 0 +# Test a query with conflicting ordering requirements (where aggregate groups are not consecutive. +# For query below aggregate groups are at indices [0, 2] and [1]) with group by. . Please note that there is group by query IIIII rowsort SELECT a, b, FIRST_VALUE(d ORDER BY c DESC), FIRST_VALUE(d ORDER BY d DESC, c DESC), FIRST_VALUE(d ORDER BY c ASC) FROM multiple_ordered_table From b813071244a3b21f7fb573cfd9fb7c9b25f6c8ee Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 7 Dec 2023 11:09:57 +0300 Subject: [PATCH 35/49] Minor changes --- .../physical-plan/src/aggregates/mod.rs | 2 +- .../src/aggregates/no_grouping.rs | 65 +++++++++++++------ 2 files changed, 47 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 76cf2450b31d..73a2e9428a58 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1049,7 +1049,7 @@ fn create_accumulators( /// returns a vector of ArrayRefs, where each entry corresponds to either the /// final value (mode = Final, FinalPartitioned and Single) or states (mode = Partial) fn finalize_aggregation( - accumulators: &[AccumulatorItem], + accumulators: &[&AccumulatorItem], mode: &AggregateMode, ) -> Result>> { match mode { diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 2c40f6741f92..6461ba059630 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -39,9 +39,23 @@ use datafusion_common::utils::get_at_indices; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_physical_expr::{LexOrdering, PhysicalExpr}; use futures::stream::{Stream, StreamExt}; +use itertools::izip; use super::AggregateExec; +/// A structure storing necessary data for aggregate expr evaluation. +/// +/// # Fields +/// +/// - `expressions`: A vector expressions that aggregate expression refers e.g for CORR(a, b) this will be a, b. +/// - `filter_expression`: A vector of optional filter expression associated with aggregate expression. +/// - `accumulator`: The accumulator used to calculate aggregate expression result. +pub struct AggregateExprData { + expressions: Vec>, + filter_expression: Option>, + accumulator: AccumulatorItem, +} + /// A structure representing an aggregate group. /// /// The `AggregateGroup` struct is all aggregate expressions @@ -51,15 +65,11 @@ use super::AggregateExec; /// /// # Fields /// -/// - `aggregate_expressions`: A vector of vectors containing aggregate expressions. -/// - `filter_expressions`: A vector of optional filter expressions associated with each aggregate expression. -/// - `accumulators`: A vector of `AccumulatorItem` instances representing accumulators for the group. +/// - `aggregates`: A vector of `AggregateExprData` which stores necessary fields for successful evaluation of the each aggregate expression. /// - `requirement`: A `LexOrdering` instance specifying the lexical ordering requirement of the group. /// - `group_indices`: A vector of indices indicating position of each aggregation in the original aggregate expression. pub struct AggregateGroup { - aggregate_expressions: Vec>>, - filter_expressions: Vec>>, - accumulators: Vec, + aggregates: Vec, requirement: LexOrdering, group_indices: Vec, } @@ -126,10 +136,21 @@ impl AggregateStream { let filter_expressions = get_at_indices(&filter_expressions, indices)?; let accumulators = create_accumulators(&aggr_exprs)?; + let aggregates = izip!( + aggregate_expressions.into_iter(), + filter_expressions.into_iter(), + accumulators.into_iter() + ) + .map(|(expressions, filter_expression, accumulator)| { + AggregateExprData { + expressions, + filter_expression, + accumulator, + } + }) + .collect::>(); Ok(AggregateGroup { - aggregate_expressions, - filter_expressions, - accumulators, + aggregates, requirement: requirement.to_vec(), group_indices: indices.to_vec(), }) @@ -261,9 +282,6 @@ fn aggregate_batch( // 1.3 evaluate expressions // 1.4 update / merge accumulators with the expressions' values - let accumulators = &mut aggregate_group.accumulators; - let expressions = &mut aggregate_group.aggregate_expressions; - let filters = &mut aggregate_group.filter_expressions; let requirement = &aggregate_group.requirement; let batch = if requirement.is_empty() { batch.clone() @@ -271,11 +289,12 @@ fn aggregate_batch( sort_batch(batch, requirement, None)? }; // 1.1 - accumulators - .iter_mut() - .zip(expressions) - .zip(filters) - .try_for_each(|((accum, expr), filter)| { + aggregate_group.aggregates.iter_mut().try_for_each( + |AggregateExprData { + expressions: expr, + filter_expression: filter, + accumulator: accum, + }| { // 1.2 let batch = match filter { Some(filter) => Cow::Owned(batch_filter(&batch, filter)?), @@ -303,7 +322,8 @@ fn aggregate_batch( let size_post = accum.size(); allocated += size_post.saturating_sub(size_pre); res - })?; + }, + )?; Ok(allocated) } @@ -316,7 +336,14 @@ fn finalize_aggregation_groups( ) -> Result> { let aggregate_group_results = aggregate_groups .iter() - .map(|aggregate_group| finalize_aggregation(&aggregate_group.accumulators, mode)) + .map(|aggregate_group| { + let accumulators = aggregate_group + .aggregates + .iter() + .map(|elem| &elem.accumulator) + .collect::>(); + finalize_aggregation(&accumulators, mode) + }) .collect::>>()?; let aggregate_group_indices = aggregate_groups .iter() From eb97d9960553e76f2b77685fdc5a15562f9e25b2 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 10:11:17 +0300 Subject: [PATCH 36/49] Fix changing tests --- datafusion/sqllogictest/test_files/aggregate.slt | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index bcda3464f49b..5fb28cb165c3 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -132,9 +132,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 statement error This feature is not implemented: LIMIT not supported in ARRAY_AGG: 1 SELECT array_agg(c13 LIMIT 1) FROM aggregate_test_100 @@ -3241,4 +3240,4 @@ select count(*) from (select count(*) from (select 1)); query I select count(*) from (select count(*) a, count(*) b from (select 1)); ---- -1 \ No newline at end of file +1 From 43c33ae5bfb8847fc4159a69f44301dca86822f7 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 13 Dec 2023 10:42:52 +0300 Subject: [PATCH 37/49] Initial review --- .../src/aggregate/array_agg_ordered.rs | 63 +++++++++---------- .../physical-expr/src/aggregate/first_last.rs | 15 ++--- 2 files changed, 33 insertions(+), 45 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 68ecec9910b7..350297b636f0 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -192,25 +192,25 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { if values.is_empty() { return Ok(()); } - let value = &values[0]; - let orderings = &values[1..]; + let value_array_ref = &values[0]; + let ordering_array_refs = &values[1..]; - let n_row = value.len(); + let num_rows = value_array_ref.len(); // Convert &[ArrayRef] to Vec> - let new_ordering_values = (0..n_row) - .map(|idx| get_row_at_idx(orderings, idx)) + let new_ordering_values = (0..num_rows) + .map(|idx| get_row_at_idx(ordering_array_refs, idx)) .collect::>>()?; // Convert ArrayRef to Vec - let new_values = (0..n_row) - .map(|idx| ScalarValue::try_from_array(value, idx)) + let new_scalar_values = (0..num_rows) + .map(|idx| ScalarValue::try_from_array(value_array_ref, idx)) .collect::>>()?; let sort_options = get_sort_options(&self.ordering_req); // Merge new values and new orderings let (merged_values, merged_ordering_values) = merge_ordered_arrays( - &[&self.values, &new_values], + &[&self.values, &new_scalar_values], &[&self.ordering_values, &new_ordering_values], &sort_options, )?; @@ -231,40 +231,35 @@ impl Accumulator for OrderSensitiveArrayAggAccumulator { let agg_orderings = &states[1]; if let Some(agg_orderings) = agg_orderings.as_list_opt::() { - // Stores ARRAY_AGG results coming from each partition - let mut partition_values = vec![]; - // Stores ordering requirement expression results coming from each partition - let mut partition_ordering_values = vec![]; - - // Existing values should be merged also. - partition_values.push(self.values.as_slice()); - partition_ordering_values.push(self.ordering_values.as_slice()); + // Stores ARRAY_AGG results coming from each partition. Existing values should be merged also. + let mut partition_values = vec![self.values.as_slice()]; let array_agg_res = ScalarValue::convert_array_to_scalar_vec(array_agg_values)?; - for v in array_agg_res.iter() { - partition_values.push(v); - } + partition_values.extend(array_agg_res.iter().map(|v| v.as_slice())); - let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)?; + // Stores ordering requirement expression results coming from each partition. Existing values should be merged also. + let mut partition_ordering_values = vec![self.ordering_values.as_slice()]; - let orderings = orderings.into_iter().map(|partition_ordering_rows| { - // Extract value from struct to ordering_rows for each group/partition - partition_ordering_rows.into_iter().map(|ordering_row| { - if let ScalarValue::Struct(Some(ordering_columns_per_row), _) = ordering_row { - Ok(ordering_columns_per_row) - } else { - exec_err!( + let orderings = ScalarValue::convert_array_to_scalar_vec(agg_orderings)? + .into_iter() + .map(|partition_ordering_rows| { + partition_ordering_rows + .into_iter() + .map(|ordering_row| match ordering_row { + ScalarValue::Struct(Some(ordering_columns_per_row), _) => + Ok(ordering_columns_per_row), + _ => exec_err!( "Expects to receive ScalarValue::Struct(Some(..), _) but got:{:?}", ordering_row.data_type() - ) - } - }).collect::>>() - }).collect::>>()?; - for ordering_value in orderings.iter() { - partition_ordering_values.push(ordering_value); - } + ), + }) + .collect::>>() + }) + .collect::>>()?; + + partition_ordering_values.extend(orderings.iter().map(|v| v.as_slice())); let sort_options = get_sort_options(&self.ordering_req); let (new_values, new_orderings) = merge_ordered_arrays( diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 1efd61505636..4f5d12f94e66 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -569,23 +569,16 @@ fn get_value_idx( ordering_req: LexOrderingRef, is_set: bool, ) -> Result> { - let value = &values[0]; - let _ordering_values = &values[1..]; - if value.is_empty() { - // For empty batches there is nothing to update + let value_array_ref = &values[0]; + // Return None for empty batches or when no ordering is specified and is_set is true. + if value_array_ref.is_empty() || (is_set && FIRST && ordering_req.is_empty()) { return Ok(None); } - if is_set && FIRST && ordering_req.is_empty() { - // No need to overwrite existing value, when no ordering is specified - // Just use first value encountered - return Ok(None); - } - Ok(Some(if FIRST { 0 } else { // LAST - value.len() - 1 + value_array_ref.len() - 1 })) } From be85af399a6691aa7bbfa7c0b0e9ff47e97db65a Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 13 Dec 2023 11:37:08 +0300 Subject: [PATCH 38/49] Partial not working commit --- .../physical-plan/src/aggregates/mod.rs | 122 ++++++++++++------ 1 file changed, 81 insertions(+), 41 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 73a2e9428a58..2aba3b8326dc 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -18,6 +18,7 @@ //! Aggregates functionalities use std::any::Any; +use std::collections::HashMap; use std::sync::Arc; use super::DisplayAs; @@ -927,50 +928,89 @@ fn get_aggregate_expr_groups( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result> { - let mut used_indices: HashSet = HashSet::new(); - let mut groups = vec![]; - while used_indices.len() != aggr_exprs.len() { - let mut group: Option<(Vec, LexOrdering)> = None; - for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { - // Group is empty and index is not already in another group. - if used_indices.contains(&idx) { - // Skip this group, it is already inserted. - continue; - } - let (mut group_indices, mut req) = if let Some((group_indices, req)) = group { - (group_indices, req) - } else { - // Initialize an empty group with empty requirement - (vec![], vec![]) - }; - if let Some(finer) = - finer_ordering(&req, aggr_expr, group_by, eq_properties, agg_mode) - { - req = finer; - group_indices.push(idx); - } else if let Some(reverse) = aggr_expr.reverse_expr() { - if let Some(finer) = - finer_ordering(&req, &reverse, group_by, eq_properties, agg_mode) - { - *aggr_expr = reverse; - req = finer; - group_indices.push(idx); - } + // let mut used_indices: HashSet = HashSet::new(); + // let mut groups = vec![]; + let mut groups_v2 : HashMap> = HashMap::new(); + for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { + let req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); + let mut sa = groups_v2.remove(&req).unwrap_or(vec![]); + if let Some(finer_ordering) = finer_ordering( + &req, + aggr_expr, + group_by, + eq_properties, + agg_mode, + ) { + sa.push(idx); + groups_v2.insert(finer_ordering, sa); + } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &req, + &reversed_expr, + group_by, + eq_properties, + agg_mode, + ) { + *aggr_expr = reversed_expr; + sa.push(idx); + groups_v2.insert(finer_ordering, sa); } - // Update group with new entries - group = Some((group_indices, req)); - } - // We cannot received None here. - let (indices, requirement) = - group.ok_or_else(|| exec_datafusion_err!("Cannot Receive empty group"))?; - used_indices.extend(indices.iter()); - groups.push(AggregateExprGroup { - indices, - requirement, - }); + }; } - Ok(groups) + + + // while used_indices.len() != aggr_exprs.len() { + // let mut current_group: Option<(Vec, LexOrdering)> = None; + // for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { + // // Group is empty and index is not already in another group. + // if used_indices.contains(&idx) { + // // Skip this group, it is already inserted. + // continue; + // } + // // Initialize an empty group with empty requirement + // let (mut group_indices, mut ordering_req) = + // current_group.unwrap_or((vec![], vec![])); + // + // if let Some(finer_ordering) = finer_ordering( + // &ordering_req, + // aggr_expr, + // group_by, + // eq_properties, + // agg_mode, + // ) { + // ordering_req = finer_ordering; + // group_indices.push(idx); + // } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { + // if let Some(finer_ordering) = finer_ordering( + // &ordering_req, + // &reversed_expr, + // group_by, + // eq_properties, + // agg_mode, + // ) { + // *aggr_expr = reversed_expr; + // ordering_req = finer_ordering; + // group_indices.push(idx); + // } + // } + // // Update group with new entries + // current_group = Some((group_indices, ordering_req)); + // } + // // We cannot received None here. + // let (indices, requirement) = current_group + // .ok_or_else(|| exec_datafusion_err!("Cannot Receive empty group"))?; + // used_indices.extend(indices.iter()); + // groups.push(AggregateExprGroup { + // indices, + // requirement, + // }); + // } + + Ok(groups_v2.into_iter().map(|(requirement, indices)| AggregateExprGroup { + indices, + requirement, + }).collect()) } /// returns physical expressions for arguments to evaluate against a batch From 74825e99761ecc5b1cb262765928f117cb14dbee Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 13:09:39 +0300 Subject: [PATCH 39/49] Simplifications --- .../physical-plan/src/aggregates/mod.rs | 112 +++++------------- 1 file changed, 31 insertions(+), 81 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2aba3b8326dc..af1c78186ecd 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -18,7 +18,6 @@ //! Aggregates functionalities use std::any::Any; -use std::collections::HashMap; use std::sync::Arc; use super::DisplayAs; @@ -39,7 +38,7 @@ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow_schema::DataType; use datafusion_common::stats::Precision; -use datafusion_common::{exec_datafusion_err, plan_err, DataFusionError, Result}; +use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ @@ -49,7 +48,6 @@ use datafusion_physical_expr::{ physical_exprs_contains, AggregateExpr, EquivalenceProperties, LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; -use hashbrown::HashSet; use itertools::Itertools; @@ -928,89 +926,41 @@ fn get_aggregate_expr_groups( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result> { - // let mut used_indices: HashSet = HashSet::new(); - // let mut groups = vec![]; - let mut groups_v2 : HashMap> = HashMap::new(); + let mut groups: Vec<(LexOrdering, Vec)> = vec![]; for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { let req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); - let mut sa = groups_v2.remove(&req).unwrap_or(vec![]); - if let Some(finer_ordering) = finer_ordering( - &req, - aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - sa.push(idx); - groups_v2.insert(finer_ordering, sa); - } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { - if let Some(finer_ordering) = finer_ordering( - &req, - &reversed_expr, - group_by, - eq_properties, - agg_mode, - ) { - *aggr_expr = reversed_expr; - sa.push(idx); - groups_v2.insert(finer_ordering, sa); + let mut group_match = false; + for (key, value) in groups.iter_mut() { + if let Some(finer_ordering) = + finer_ordering(key, aggr_expr, group_by, eq_properties, agg_mode) + { + value.push(idx); + *key = finer_ordering; + group_match = true; + } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = + finer_ordering(key, &reversed_expr, group_by, eq_properties, agg_mode) + { + *aggr_expr = reversed_expr; + value.push(idx); + *key = finer_ordering; + group_match = true; + } } - }; + } + if !group_match { + // there is no existing group that matches. Insert new group + groups.push((req, vec![idx])); + } } - - - // while used_indices.len() != aggr_exprs.len() { - // let mut current_group: Option<(Vec, LexOrdering)> = None; - // for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { - // // Group is empty and index is not already in another group. - // if used_indices.contains(&idx) { - // // Skip this group, it is already inserted. - // continue; - // } - // // Initialize an empty group with empty requirement - // let (mut group_indices, mut ordering_req) = - // current_group.unwrap_or((vec![], vec![])); - // - // if let Some(finer_ordering) = finer_ordering( - // &ordering_req, - // aggr_expr, - // group_by, - // eq_properties, - // agg_mode, - // ) { - // ordering_req = finer_ordering; - // group_indices.push(idx); - // } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { - // if let Some(finer_ordering) = finer_ordering( - // &ordering_req, - // &reversed_expr, - // group_by, - // eq_properties, - // agg_mode, - // ) { - // *aggr_expr = reversed_expr; - // ordering_req = finer_ordering; - // group_indices.push(idx); - // } - // } - // // Update group with new entries - // current_group = Some((group_indices, ordering_req)); - // } - // // We cannot received None here. - // let (indices, requirement) = current_group - // .ok_or_else(|| exec_datafusion_err!("Cannot Receive empty group"))?; - // used_indices.extend(indices.iter()); - // groups.push(AggregateExprGroup { - // indices, - // requirement, - // }); - // } - - Ok(groups_v2.into_iter().map(|(requirement, indices)| AggregateExprGroup { - indices, - requirement, - }).collect()) + Ok(groups + .into_iter() + .map(|(requirement, indices)| AggregateExprGroup { + indices, + requirement, + }) + .collect()) } /// returns physical expressions for arguments to evaluate against a batch From 9703390547205b5b108a60624f588f56abc253d1 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 13 Dec 2023 13:17:05 +0300 Subject: [PATCH 40/49] Update mod.rs --- .../physical-plan/src/aggregates/mod.rs | 159 +++++++----------- 1 file changed, 64 insertions(+), 95 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2aba3b8326dc..11697eb4e32e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -18,7 +18,6 @@ //! Aggregates functionalities use std::any::Any; -use std::collections::HashMap; use std::sync::Arc; use super::DisplayAs; @@ -857,12 +856,17 @@ fn get_aggregate_expr_req( group_by: &PhysicalGroupBy, agg_mode: &AggregateMode, ) -> LexOrdering { - let mut req = aggr_expr.order_bys().unwrap_or_default().to_vec(); // If // - aggregation function is not order-sensitive and // - aggregation is performing a "second stage" calculation, and // - all aggregate function requirement is inside group by expression // 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. @@ -877,9 +881,6 @@ fn get_aggregate_expr_req( !physical_exprs_contains(&physical_exprs, &sort_expr.expr) }); } - if !is_order_sensitive(aggr_expr) || !agg_mode.is_first_stage() { - req.clear(); - } req } @@ -928,89 +929,55 @@ fn get_aggregate_expr_groups( eq_properties: &EquivalenceProperties, agg_mode: &AggregateMode, ) -> Result> { - // let mut used_indices: HashSet = HashSet::new(); - // let mut groups = vec![]; - let mut groups_v2 : HashMap> = HashMap::new(); - for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { - let req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); - let mut sa = groups_v2.remove(&req).unwrap_or(vec![]); - if let Some(finer_ordering) = finer_ordering( - &req, - aggr_expr, - group_by, - eq_properties, - agg_mode, - ) { - sa.push(idx); - groups_v2.insert(finer_ordering, sa); - } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { + let mut used_indices: HashSet = HashSet::new(); + let mut groups = vec![]; + while used_indices.len() != aggr_exprs.len() { + let mut current_group: Option<(Vec, LexOrdering)> = None; + for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { + // Group is empty and index is not already in another group. + if used_indices.contains(&idx) { + // Skip this group, it is already inserted. + continue; + } + // Initialize an empty group with empty requirement + let (mut group_indices, mut ordering_req) = + current_group.unwrap_or((vec![], vec![])); + if let Some(finer_ordering) = finer_ordering( - &req, - &reversed_expr, + &ordering_req, + aggr_expr, group_by, eq_properties, agg_mode, ) { - *aggr_expr = reversed_expr; - sa.push(idx); - groups_v2.insert(finer_ordering, sa); + ordering_req = finer_ordering; + group_indices.push(idx); + } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { + if let Some(finer_ordering) = finer_ordering( + &ordering_req, + &reversed_expr, + group_by, + eq_properties, + agg_mode, + ) { + *aggr_expr = reversed_expr; + ordering_req = finer_ordering; + group_indices.push(idx); + } } - }; + // Update group with new entries + current_group = Some((group_indices, ordering_req)); + } + // We cannot received None here. + let (indices, requirement) = current_group + .ok_or_else(|| exec_datafusion_err!("Cannot Receive empty group"))?; + used_indices.extend(indices.iter()); + groups.push(AggregateExprGroup { + indices, + requirement, + }); } - - - - // while used_indices.len() != aggr_exprs.len() { - // let mut current_group: Option<(Vec, LexOrdering)> = None; - // for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { - // // Group is empty and index is not already in another group. - // if used_indices.contains(&idx) { - // // Skip this group, it is already inserted. - // continue; - // } - // // Initialize an empty group with empty requirement - // let (mut group_indices, mut ordering_req) = - // current_group.unwrap_or((vec![], vec![])); - // - // if let Some(finer_ordering) = finer_ordering( - // &ordering_req, - // aggr_expr, - // group_by, - // eq_properties, - // agg_mode, - // ) { - // ordering_req = finer_ordering; - // group_indices.push(idx); - // } else if let Some(reversed_expr) = aggr_expr.reverse_expr() { - // if let Some(finer_ordering) = finer_ordering( - // &ordering_req, - // &reversed_expr, - // group_by, - // eq_properties, - // agg_mode, - // ) { - // *aggr_expr = reversed_expr; - // ordering_req = finer_ordering; - // group_indices.push(idx); - // } - // } - // // Update group with new entries - // current_group = Some((group_indices, ordering_req)); - // } - // // We cannot received None here. - // let (indices, requirement) = current_group - // .ok_or_else(|| exec_datafusion_err!("Cannot Receive empty group"))?; - // used_indices.extend(indices.iter()); - // groups.push(AggregateExprGroup { - // indices, - // requirement, - // }); - // } - - Ok(groups_v2.into_iter().map(|(requirement, indices)| AggregateExprGroup { - indices, - requirement, - }).collect()) + Ok(groups) } /// returns physical expressions for arguments to evaluate against a batch @@ -1217,24 +1184,26 @@ fn reorder_aggregate_expr_results( aggregate_group_results: Vec>>, aggregate_group_indices: Vec>, ) -> Vec { - let n_aggregate = aggregate_group_indices - .iter() - .map(|group_indices| group_indices.len()) - .sum(); + // Calculate the total number of aggregate results. + let n_aggregate = aggregate_group_indices.iter().flatten().count(); + // Initialize a result vector with empty vectors, one for each aggregate result. let mut result = vec![vec![]; n_aggregate]; - // Insert each aggregation result inside an aggregation group, to the proper places in the result - for (aggregate_group_result, group_indices) in aggregate_group_results + // Process each aggregate group result and its corresponding indices. + // We combine the results and indices, flattening them for processing. + aggregate_group_results .into_iter() .zip(aggregate_group_indices.iter()) - { - group_indices - .iter() - .zip(aggregate_group_result.into_iter()) - .for_each(|(&idx, aggr_state)| { - result[idx] = aggr_state; - }) - } + .flat_map(|(group_result, group_indices)| { + // Pair each group's result with its target index. + group_indices.iter().zip(group_result) + }) + .for_each(|(&idx, aggr_state)| { + // Place each aggregate state in its correct position in the result vector. + result[idx] = aggr_state; + }); + + // Flatten the result vectors into a single vector and return. result.into_iter().flatten().collect() } From f58b760e2b03b62906fa56ea6e3ec1a3e7a6489a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 13:30:43 +0300 Subject: [PATCH 41/49] Minor changes --- 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 af1c78186ecd..55c683398d05 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -928,7 +928,7 @@ fn get_aggregate_expr_groups( ) -> Result> { let mut groups: Vec<(LexOrdering, Vec)> = vec![]; for (idx, aggr_expr) in aggr_exprs.iter_mut().enumerate() { - let req = aggr_expr.order_bys().unwrap_or(&[]).to_vec(); + let req = get_aggregate_expr_req(aggr_expr, group_by, agg_mode); let mut group_match = false; for (key, value) in groups.iter_mut() { if let Some(finer_ordering) = From 585fdc8d7907d4e7fdce15b7499f151eb7dcc0f7 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Wed, 13 Dec 2023 13:38:09 +0300 Subject: [PATCH 42/49] No grouping refactor --- .../src/aggregates/no_grouping.rs | 212 ++++++++++-------- 1 file changed, 124 insertions(+), 88 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/no_grouping.rs b/datafusion/physical-plan/src/aggregates/no_grouping.rs index 6461ba059630..be379a6fa374 100644 --- a/datafusion/physical-plan/src/aggregates/no_grouping.rs +++ b/datafusion/physical-plan/src/aggregates/no_grouping.rs @@ -100,29 +100,32 @@ struct AggregateStreamInner { impl AggregateStream { /// Create a new AggregateStream pub fn new( - agg: &AggregateExec, + aggregate_exec: &AggregateExec, context: Arc, partition: usize, ) -> Result { - let agg_schema = Arc::clone(&agg.schema); - let agg_filter_expr = agg.filter_expr.clone(); + let agg_schema = Arc::clone(&aggregate_exec.schema); + let agg_filter_expr = aggregate_exec.filter_expr.clone(); - let baseline_metrics = BaselineMetrics::new(&agg.metrics, partition); - let input = agg.input.execute(partition, Arc::clone(&context))?; + let baseline_metrics = BaselineMetrics::new(&aggregate_exec.metrics, partition); + let input = aggregate_exec + .input + .execute(partition, Arc::clone(&context))?; - let aggregate_expressions = aggregate_expressions(&agg.aggr_expr, &agg.mode, 0)?; - let filter_expressions = match agg.mode { + let aggregate_expressions = + aggregate_expressions(&aggregate_exec.aggr_expr, &aggregate_exec.mode, 0)?; + let filter_expressions = match aggregate_exec.mode { AggregateMode::Partial | AggregateMode::Single | AggregateMode::SinglePartitioned => agg_filter_expr, AggregateMode::Final | AggregateMode::FinalPartitioned => { - vec![None; agg.aggr_expr.len()] + vec![None; aggregate_exec.aggr_expr.len()] } }; let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); - let aggregate_groups = agg + let aggregate_groups = aggregate_exec .aggregate_groups .iter() .map( @@ -130,7 +133,7 @@ impl AggregateStream { indices, requirement, }| { - let aggr_exprs = get_at_indices(&agg.aggr_expr, indices)?; + let aggr_exprs = get_at_indices(&aggregate_exec.aggr_expr, indices)?; let aggregate_expressions = get_at_indices(&aggregate_expressions, indices)?; let filter_expressions = @@ -158,72 +161,13 @@ impl AggregateStream { ) .collect::>>()?; - let inner = AggregateStreamInner { - schema: Arc::clone(&agg.schema), - mode: agg.mode, + let stream = create_aggregate_stream( + aggregate_exec, input, baseline_metrics, aggregate_groups, reservation, - finished: false, - }; - let stream = futures::stream::unfold(inner, |mut this| async move { - if this.finished { - return None; - } - - let elapsed_compute = this.baseline_metrics.elapsed_compute(); - - loop { - let result = match this.input.next().await { - Some(Ok(batch)) => { - let timer = elapsed_compute.timer(); - let result = aggregate_batch_groups( - &this.mode, - batch, - &mut this.aggregate_groups, - ); - - timer.done(); - - // allocate memory - // This happens AFTER we actually used the memory, but simplifies the whole accounting and we are OK with - // overshooting a bit. Also this means we either store the whole record batch or not. - match result - .and_then(|allocated| this.reservation.try_grow(allocated)) - { - Ok(_) => continue, - Err(e) => Err(e), - } - } - Some(Err(e)) => Err(e), - None => { - this.finished = true; - let timer = this.baseline_metrics.elapsed_compute().timer(); - let result = finalize_aggregation_groups( - &this.aggregate_groups, - &this.mode, - ) - .and_then(|columns| { - RecordBatch::try_new(this.schema.clone(), columns) - .map_err(Into::into) - }) - .record_output(&this.baseline_metrics); - - timer.done(); - - result - } - }; - - this.finished = true; - return Some((result, this)); - } - }); - - // seems like some consumers call this stream even after it returned `None`, so let's fuse the stream. - let stream = stream.fuse(); - let stream = Box::pin(stream); + )?; Ok(Self { schema: agg_schema, @@ -232,6 +176,96 @@ impl AggregateStream { } } +/// Creates a stream for processing aggregate expressions. +/// +/// This function constructs a stream that processes batches from `input`, aggregates +/// them according to `aggregate_groups`, and finally yields the aggregated results. +/// It handles the aggregation logic depending on the aggregate mode defined in `agg`. +/// The function also accounts for memory consumption during aggregation using `reservation`. +/// +/// # Parameters +/// - `aggregate_exec`: Reference to the `AggregateExec` struct which contains the aggregate execution plan. +/// - `input`: Stream of `RecordBatch` items representing the input data. +/// - `baseline_metrics`: Metrics for tracking the performance and resource usage. +/// - `aggregate_groups`: A vector of `AggregateGroup` structs, each representing a group of +/// aggregate expressions along with their corresponding indices and ordering requirements. +/// - `reservation`: Memory reservation handle for managing memory consumption during aggregation. +/// +/// # Returns +/// A `Result` containing the constructed stream if successful, or an error if the stream +/// creation fails. The stream yields `RecordBatch` items, each representing a batch of aggregated results. +fn create_aggregate_stream( + aggregate_exec: &AggregateExec, + input: SendableRecordBatchStream, + baseline_metrics: BaselineMetrics, + aggregate_groups: Vec, + reservation: MemoryReservation, +) -> Result>> { + let inner = AggregateStreamInner { + schema: Arc::clone(&aggregate_exec.schema), + mode: aggregate_exec.mode, + input, + baseline_metrics, + aggregate_groups, + reservation, + finished: false, + }; + let stream = futures::stream::unfold(inner, |mut this| async move { + if this.finished { + return None; + } + + let elapsed_compute = this.baseline_metrics.elapsed_compute(); + + loop { + let result = match this.input.next().await { + Some(Ok(batch)) => { + let timer = elapsed_compute.timer(); + let result = aggregate_batch_groups( + &this.mode, + batch, + &mut this.aggregate_groups, + ); + + timer.done(); + + // allocate memory + // This happens AFTER we actually used the memory, but simplifies the whole accounting and we are OK with + // overshooting a bit. Also this means we either store the whole record batch or not. + match result + .and_then(|allocated| this.reservation.try_grow(allocated)) + { + Ok(_) => continue, + Err(e) => Err(e), + } + } + Some(Err(e)) => Err(e), + None => { + this.finished = true; + let timer = this.baseline_metrics.elapsed_compute().timer(); + let result = + finalize_aggregation_groups(&this.aggregate_groups, &this.mode) + .and_then(|columns| { + RecordBatch::try_new(this.schema.clone(), columns) + .map_err(Into::into) + }) + .record_output(&this.baseline_metrics); + + timer.done(); + + result + } + }; + + this.finished = true; + return Some((result, this)); + } + }); + + // seems like some consumers call this stream even after it returned `None`, so let's fuse the stream. + Ok(Box::pin(stream.fuse())) +} + impl Stream for AggregateStream { type Item = Result; @@ -283,43 +317,45 @@ fn aggregate_batch( // 1.4 update / merge accumulators with the expressions' values let requirement = &aggregate_group.requirement; - let batch = if requirement.is_empty() { - batch.clone() + let sorted_or_original_batch = if requirement.is_empty() { + Cow::Borrowed(batch) } else { - sort_batch(batch, requirement, None)? + Cow::Owned(sort_batch(batch, requirement, None)?) }; // 1.1 aggregate_group.aggregates.iter_mut().try_for_each( |AggregateExprData { - expressions: expr, - filter_expression: filter, - accumulator: accum, + expressions, + filter_expression, + accumulator, }| { // 1.2 - let batch = match filter { - Some(filter) => Cow::Owned(batch_filter(&batch, filter)?), - None => Cow::Borrowed(&batch), + let filtered_or_original_batch = match filter_expression { + Some(filter) => { + Cow::Owned(batch_filter(&sorted_or_original_batch, filter)?) + } + None => Cow::Borrowed(&*sorted_or_original_batch), }; // 1.3 - let values = &expr + let values = &expressions .iter() .map(|e| { - e.evaluate(&batch) - .and_then(|v| v.into_array(batch.num_rows())) + e.evaluate(&filtered_or_original_batch) + .and_then(|v| v.into_array(filtered_or_original_batch.num_rows())) }) .collect::>>()?; // 1.4 - let size_pre = accum.size(); + let size_pre = accumulator.size(); let res = match mode { AggregateMode::Partial | AggregateMode::Single - | AggregateMode::SinglePartitioned => accum.update_batch(values), + | AggregateMode::SinglePartitioned => accumulator.update_batch(values), AggregateMode::Final | AggregateMode::FinalPartitioned => { - accum.merge_batch(values) + accumulator.merge_batch(values) } }; - let size_post = accum.size(); + let size_post = accumulator.size(); allocated += size_post.saturating_sub(size_pre); res }, From 184865ed556ab5edeee52b8f00ffb3e853ae198f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 15:05:21 +0300 Subject: [PATCH 43/49] Minor changes --- datafusion/physical-plan/src/aggregates/mod.rs | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3168b7889aec..6fadfd9c78f9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -340,9 +340,13 @@ impl AggregateExec { group_by.contains_null(), mode, )?; - let eq_properties = input.equivalence_properties(); - let aggregate_groups = - get_aggregate_expr_groups(&mut aggr_expr, &group_by, &eq_properties, &mode)?; + let input_eq_properties = input.equivalence_properties(); + let aggregate_groups = get_aggregate_expr_groups( + &mut aggr_expr, + &group_by, + &input_eq_properties, + &mode, + )?; let schema = Arc::new(materialize_dict_group_keys( &original_schema, @@ -381,9 +385,8 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let aggregate_eqs = input - .equivalence_properties() - .project(&projection_mapping, schema.clone()); + let aggregate_eqs = + input_eq_properties.project(&projection_mapping, schema.clone()); let output_ordering = aggregate_eqs.oeq_class().output_ordering(); Ok(AggregateExec { @@ -2180,8 +2183,6 @@ mod tests { )) as _ }) .collect::>(); - // let res = - // get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; let group_by = PhysicalGroupBy::new_single(vec![]); let res = get_aggregate_expr_groups( &mut aggr_exprs, From b5a250cdb1c7018922d7a25a94bdc1790b33e188 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 17:08:28 +0300 Subject: [PATCH 44/49] Minor changes --- .../aggregate_statistics.rs | 12 --------- .../combine_partial_final_agg.rs | 4 --- .../enforce_distribution.rs | 4 --- .../limited_distinct_aggregation.rs | 25 +++---------------- .../core/src/physical_optimizer/test_utils.rs | 1 - .../physical_optimizer/topk_aggregation.rs | 1 - datafusion/core/src/physical_planner.rs | 5 +--- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 2 -- .../physical-plan/src/aggregates/mod.rs | 18 ++----------- datafusion/physical-plan/src/limit.rs | 1 - datafusion/proto/src/physical_plan/mod.rs | 14 ----------- .../tests/cases/roundtrip_physical_plan.rs | 2 -- 12 files changed, 7 insertions(+), 82 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs index 795857b10ef5..86a8cdb7b3d4 100644 --- a/datafusion/core/src/physical_optimizer/aggregate_statistics.rs +++ b/datafusion/core/src/physical_optimizer/aggregate_statistics.rs @@ -397,7 +397,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -407,7 +406,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -429,7 +427,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -439,7 +436,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -460,7 +456,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -473,7 +468,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(coalesce), Arc::clone(&schema), )?; @@ -494,7 +488,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], source, Arc::clone(&schema), )?; @@ -507,7 +500,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(coalesce), Arc::clone(&schema), )?; @@ -539,7 +531,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], filter, Arc::clone(&schema), )?; @@ -549,7 +540,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; @@ -586,7 +576,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], filter, Arc::clone(&schema), )?; @@ -596,7 +585,6 @@ pub(crate) mod tests { PhysicalGroupBy::default(), vec![agg.count_expr()], vec![None], - vec![None], Arc::new(partial_agg), Arc::clone(&schema), )?; diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 0948445de20d..c50ea36b68ec 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -91,7 +91,6 @@ impl PhysicalOptimizerRule for CombinePartialFinalAggregate { input_agg_exec.group_by().clone(), input_agg_exec.aggr_expr().to_vec(), input_agg_exec.filter_expr().to_vec(), - input_agg_exec.order_by_expr().to_vec(), input_agg_exec.input().clone(), input_agg_exec.input_schema(), ) @@ -277,7 +276,6 @@ mod tests { group_by, aggr_expr, vec![], - vec![], input, schema, ) @@ -297,7 +295,6 @@ mod tests { group_by, aggr_expr, vec![], - vec![], input, schema, ) @@ -458,7 +455,6 @@ mod tests { final_group_by, aggr_expr, vec![], - vec![], partial_agg, schema, ) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3aed6555f305..e24fd4e82fc7 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -521,7 +521,6 @@ fn reorder_aggregate_keys( new_partial_group_by, agg_exec.aggr_expr().to_vec(), agg_exec.filter_expr().to_vec(), - agg_exec.order_by_expr().to_vec(), agg_exec.input().clone(), agg_exec.input_schema.clone(), )?)) @@ -548,7 +547,6 @@ fn reorder_aggregate_keys( new_group_by, agg_exec.aggr_expr().to_vec(), agg_exec.filter_expr().to_vec(), - agg_exec.order_by_expr().to_vec(), partial_agg, agg_exec.input_schema(), )?); @@ -1909,14 +1907,12 @@ pub(crate) mod tests { final_grouping, vec![], vec![], - vec![], Arc::new( AggregateExec::try_new( AggregateMode::Partial, group_by, vec![], vec![], - vec![], input, schema.clone(), ) diff --git a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs index 8f5dbc2e9214..540f9a6a132b 100644 --- a/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/limited_distinct_aggregation.rs @@ -55,7 +55,6 @@ impl LimitedDistinctAggregation { aggr.group_by().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), - aggr.order_by_expr().to_vec(), aggr.input().clone(), aggr.input_schema(), ) @@ -307,7 +306,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -316,7 +314,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ Arc::new(partial_agg), /* input */ schema.clone(), /* input_schema */ )?; @@ -359,7 +356,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -401,7 +397,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -443,7 +438,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string(), "b".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -452,7 +446,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ Arc::new(group_by_agg), /* input */ schema.clone(), /* input_schema */ )?; @@ -495,7 +488,6 @@ mod tests { build_group_by(&schema.clone(), vec![]), vec![], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -526,7 +518,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![agg.count_expr()], /* aggr_expr */ vec![None], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -563,7 +554,6 @@ mod tests { build_group_by(&schema.clone(), vec!["a".to_string()]), vec![], /* aggr_expr */ vec![filter_expr], /* filter_expr */ - vec![None], /* order_by_expr */ source, /* input */ schema.clone(), /* input_schema */ )?; @@ -592,22 +582,15 @@ mod tests { let source = parquet_exec_with_sort(vec![sort_key]); let schema = source.schema(); - // `SELECT a FROM MemoryExec GROUP BY a ORDER BY a LIMIT 10;`, Single AggregateExec - let order_by_expr = Some(vec![PhysicalSortExpr { - expr: expressions::col("a", &schema.clone()).unwrap(), - options: SortOptions::default(), - }]); - // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec // the `a > 1` filter is applied in the AggregateExec let single_agg = AggregateExec::try_new( AggregateMode::Single, build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![None], /* filter_expr */ - vec![order_by_expr], /* order_by_expr */ - source, /* input */ - schema.clone(), /* input_schema */ + vec![], /* aggr_expr */ + vec![None], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ )?; let limit_exec = LocalLimitExec::new( Arc::new(single_agg), diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 37a76eff1ee2..678dc1f373e3 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -339,7 +339,6 @@ pub fn aggregate_exec(input: Arc) -> Arc { PhysicalGroupBy::default(), vec![], vec![], - vec![], input, schema, ) diff --git a/datafusion/core/src/physical_optimizer/topk_aggregation.rs b/datafusion/core/src/physical_optimizer/topk_aggregation.rs index 52d34d4f8198..dd0261420304 100644 --- a/datafusion/core/src/physical_optimizer/topk_aggregation.rs +++ b/datafusion/core/src/physical_optimizer/topk_aggregation.rs @@ -73,7 +73,6 @@ impl TopKAggregation { aggr.group_by().clone(), aggr.aggr_expr().to_vec(), aggr.filter_expr().to_vec(), - aggr.order_by_expr().to_vec(), aggr.input().clone(), aggr.input_schema(), ) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index ab38b3ec6d2f..364fea376add 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -799,14 +799,13 @@ impl DefaultPhysicalPlanner { }) .collect::>>()?; - let (aggregates, filters, order_bys) : (Vec<_>, Vec<_>, Vec<_>) = multiunzip(agg_filter); + let (aggregates, filters, _order_bys) : (Vec<_>, Vec<_>, Vec<_>) = multiunzip(agg_filter); let initial_aggr = Arc::new(AggregateExec::try_new( AggregateMode::Partial, groups.clone(), aggregates.clone(), filters.clone(), - order_bys, input_exec, physical_input_schema.clone(), )?); @@ -824,7 +823,6 @@ impl DefaultPhysicalPlanner { // To reflect such changes to subsequent stages, use the updated // `AggregateExpr`/`PhysicalSortExpr` objects. let updated_aggregates = initial_aggr.aggr_expr().to_vec(); - let updated_order_bys = initial_aggr.order_by_expr().to_vec(); let next_partition_mode = if can_repartition { // construct a second aggregation with 'AggregateMode::FinalPartitioned' @@ -848,7 +846,6 @@ impl DefaultPhysicalPlanner { final_grouping_set, updated_aggregates, filters, - updated_order_bys, initial_aggr, physical_input_schema.clone(), )?)) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 821f236af87b..9069dbbd5850 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -109,7 +109,6 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str group_by.clone(), aggregate_expr.clone(), vec![None], - vec![None], running_source, schema.clone(), ) @@ -122,7 +121,6 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str group_by.clone(), aggregate_expr.clone(), vec![None], - vec![None], usual_source, schema.clone(), ) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6fadfd9c78f9..74c63cf372ec 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -290,8 +290,6 @@ pub struct AggregateExec { aggr_expr: Vec>, /// FILTER (WHERE clause) expression for each aggregate expression filter_expr: Vec>>, - /// (ORDER BY clause) expression for each aggregate expression - order_by_expr: Vec>, /// Stores aggregate groups where each group has different ordering requirement. aggregate_groups: Vec, /// Set if the output of this aggregation is truncated by a upstream sort/limit clause @@ -328,8 +326,6 @@ impl AggregateExec { group_by: PhysicalGroupBy, mut aggr_expr: Vec>, filter_expr: Vec>>, - // Ordering requirement of each aggregate expression - order_by_expr: Vec>, input: Arc, input_schema: SchemaRef, ) -> Result { @@ -394,7 +390,6 @@ impl AggregateExec { group_by, aggr_expr, filter_expr, - order_by_expr, aggregate_groups, input, original_schema, @@ -441,7 +436,8 @@ impl AggregateExec { /// ORDER BY clause expression for each aggregate expression pub fn order_by_expr(&self) -> &[Option] { - &self.order_by_expr + // &self.order_by_expr + &[] } /// Input plan @@ -710,7 +706,6 @@ impl ExecutionPlan for AggregateExec { self.group_by.clone(), self.aggr_expr.clone(), self.filter_expr.clone(), - self.order_by_expr.clone(), children[0].clone(), self.input_schema.clone(), )?; @@ -1397,7 +1392,6 @@ mod tests { grouping_set.clone(), aggregates.clone(), vec![None], - vec![None], input, input_schema.clone(), )?); @@ -1476,7 +1470,6 @@ mod tests { final_grouping_set, aggregates, vec![None], - vec![None], merge, input_schema, )?); @@ -1542,7 +1535,6 @@ mod tests { grouping_set.clone(), aggregates.clone(), vec![None], - vec![None], input, input_schema.clone(), )?); @@ -1590,7 +1582,6 @@ mod tests { final_grouping_set, aggregates, vec![None], - vec![None], merge, input_schema, )?); @@ -1857,7 +1848,6 @@ mod tests { groups, aggregates, vec![None; 3], - vec![None; 3], input.clone(), input_schema.clone(), )?); @@ -1913,7 +1903,6 @@ mod tests { groups.clone(), aggregates.clone(), vec![None], - vec![None], blocking_exec, schema, )?); @@ -1952,7 +1941,6 @@ mod tests { groups, aggregates.clone(), vec![None], - vec![None], blocking_exec, schema, )?); @@ -2054,7 +2042,6 @@ mod tests { groups.clone(), aggregates.clone(), vec![None], - vec![Some(ordering_req.clone())], memory_exec, schema.clone(), )?); @@ -2070,7 +2057,6 @@ mod tests { groups, aggregates.clone(), vec![None], - vec![Some(ordering_req)], coalesce, schema, )?) as Arc; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 355561c36f35..37e8ffd76159 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -878,7 +878,6 @@ mod tests { build_group_by(&csv.schema().clone(), vec!["i".to_string()]), vec![], vec![None], - vec![None], csv.clone(), csv.schema().clone(), )?; diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 73091a6fced9..359ff67821d8 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -427,19 +427,6 @@ impl AsExecutionPlan for PhysicalPlanNode { .transpose() }) .collect::, _>>()?; - let physical_order_by_expr = hash_agg - .order_by_expr - .iter() - .map(|expr| { - expr.sort_expr - .iter() - .map(|e| { - parse_physical_sort_expr(e, registry, &physical_schema) - }) - .collect::>>() - .map(|exprs| (!exprs.is_empty()).then_some(exprs)) - }) - .collect::>>()?; let physical_aggr_expr: Vec> = hash_agg .aggr_expr @@ -498,7 +485,6 @@ impl AsExecutionPlan for PhysicalPlanNode { PhysicalGroupBy::new(group_expr, null_expr, groups), physical_aggr_expr, physical_filter_expr, - physical_order_by_expr, input, Arc::new(input_schema.try_into()?), )?)) diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index da76209dbb49..621fc6bece9a 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -311,7 +311,6 @@ fn rountrip_aggregate() -> Result<()> { PhysicalGroupBy::new_single(groups.clone()), aggregates.clone(), vec![None], - vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, )?)) @@ -379,7 +378,6 @@ fn roundtrip_aggregate_udaf() -> Result<()> { PhysicalGroupBy::new_single(groups.clone()), aggregates.clone(), vec![None], - vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, )?), From 77b5c065e126f554c7821f24f727174c58734979 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 17:12:17 +0300 Subject: [PATCH 45/49] Minor changes --- datafusion/proto/tests/cases/roundtrip_physical_plan.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs index 621fc6bece9a..4a512413e73e 100644 --- a/datafusion/proto/tests/cases/roundtrip_physical_plan.rs +++ b/datafusion/proto/tests/cases/roundtrip_physical_plan.rs @@ -592,7 +592,6 @@ fn roundtrip_distinct_count() -> Result<()> { PhysicalGroupBy::new_single(groups), aggregates.clone(), vec![None], - vec![None], Arc::new(EmptyExec::new(schema.clone())), schema, )?)) From 86cf0b9b9e512fc5e28021afef3275659e126fed Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 17:19:42 +0300 Subject: [PATCH 46/49] instead of input use equivalence --- datafusion/physical-plan/src/aggregates/mod.rs | 4 ++-- .../physical-plan/src/windows/bounded_window_agg_exec.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 6 ++---- datafusion/physical-plan/src/windows/window_agg_exec.rs | 6 ++++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 74c63cf372ec..6af84c6e8de6 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -343,7 +343,6 @@ impl AggregateExec { &input_eq_properties, &mode, )?; - let schema = Arc::new(materialize_dict_group_keys( &original_schema, group_by.expr.len(), @@ -355,7 +354,8 @@ impl AggregateExec { // 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_eq_properties); let mut new_requirement = indices .iter() .map(|&idx| PhysicalSortRequirement { diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 431a43bc6055..ef4ea92f7e46 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -108,7 +108,7 @@ impl BoundedWindowAggExec { InputOrderMode::Sorted => { let indices = get_ordered_partition_by_indices( window_expr[0].partition_by(), - &input, + &input.equivalence_properties(), ); if indices.len() == partition_by_exprs.len() { indices diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 3187e6b0fbd3..38bce539a8f7 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -319,11 +319,9 @@ pub(crate) fn calc_requirements< // resulting vector (a, b) is a preset of the existing ordering (a, b, c). pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], - input: &Arc, + input_eq_properties: &EquivalenceProperties, ) -> Vec { - let (_, indices) = input - .equivalence_properties() - .find_longest_permutation(partition_by_exprs); + let (_, indices) = input_eq_properties.find_longest_permutation(partition_by_exprs); indices } diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 6c245f65ba4f..a9efe4a3c0c8 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -80,8 +80,10 @@ impl WindowAggExec { let schema = create_schema(&input.schema(), &window_expr)?; let schema = Arc::new(schema); - let ordered_partition_by_indices = - get_ordered_partition_by_indices(window_expr[0].partition_by(), &input); + let ordered_partition_by_indices = get_ordered_partition_by_indices( + window_expr[0].partition_by(), + &input.equivalence_properties(), + ); Ok(Self { input, window_expr, From 2ce128d949d66b584d297c0dd677c6e58d75d9bd Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 13 Dec 2023 18:18:38 +0300 Subject: [PATCH 47/49] Update proto --- datafusion/physical-plan/src/aggregates/mod.rs | 8 +------- datafusion/proto/proto/datafusion.proto | 1 - datafusion/proto/src/generated/pbjson.rs | 18 ------------------ datafusion/proto/src/generated/prost.rs | 2 -- datafusion/proto/src/physical_plan/mod.rs | 7 ------- 5 files changed, 1 insertion(+), 35 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 6af84c6e8de6..b318aab25873 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -434,12 +434,6 @@ impl AggregateExec { &self.filter_expr } - /// ORDER BY clause expression for each aggregate expression - pub fn order_by_expr(&self) -> &[Option] { - // &self.order_by_expr - &[] - } - /// Input plan pub fn input(&self) -> &Arc { &self.input @@ -517,7 +511,7 @@ impl AggregateExec { return false; } // ensure there are no order by expressions - if self.order_by_expr().iter().any(|e| e.is_some()) { + if self.aggr_expr().iter().any(|e| e.order_bys().is_some()) { return false; } // ensure there is no output ordering; can this rule be relaxed? diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index f391592dfe76..bd8053c817e7 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1553,7 +1553,6 @@ message AggregateExecNode { repeated PhysicalExprNode null_expr = 8; repeated bool groups = 9; repeated MaybeFilter filter_expr = 10; - repeated MaybePhysicalSortExprs order_by_expr = 11; } message GlobalLimitExecNode { diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index d506b5dcce53..88310be0318a 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -36,9 +36,6 @@ impl serde::Serialize for AggregateExecNode { if !self.filter_expr.is_empty() { len += 1; } - if !self.order_by_expr.is_empty() { - len += 1; - } let mut struct_ser = serializer.serialize_struct("datafusion.AggregateExecNode", len)?; if !self.group_expr.is_empty() { struct_ser.serialize_field("groupExpr", &self.group_expr)?; @@ -72,9 +69,6 @@ impl serde::Serialize for AggregateExecNode { if !self.filter_expr.is_empty() { struct_ser.serialize_field("filterExpr", &self.filter_expr)?; } - if !self.order_by_expr.is_empty() { - struct_ser.serialize_field("orderByExpr", &self.order_by_expr)?; - } struct_ser.end() } } @@ -102,8 +96,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { "groups", "filter_expr", "filterExpr", - "order_by_expr", - "orderByExpr", ]; #[allow(clippy::enum_variant_names)] @@ -118,7 +110,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { NullExpr, Groups, FilterExpr, - OrderByExpr, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -150,7 +141,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { "nullExpr" | "null_expr" => Ok(GeneratedField::NullExpr), "groups" => Ok(GeneratedField::Groups), "filterExpr" | "filter_expr" => Ok(GeneratedField::FilterExpr), - "orderByExpr" | "order_by_expr" => Ok(GeneratedField::OrderByExpr), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -180,7 +170,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { let mut null_expr__ = None; let mut groups__ = None; let mut filter_expr__ = None; - let mut order_by_expr__ = None; while let Some(k) = map_.next_key()? { match k { GeneratedField::GroupExpr => { @@ -243,12 +232,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { } filter_expr__ = Some(map_.next_value()?); } - GeneratedField::OrderByExpr => { - if order_by_expr__.is_some() { - return Err(serde::de::Error::duplicate_field("orderByExpr")); - } - order_by_expr__ = Some(map_.next_value()?); - } } } Ok(AggregateExecNode { @@ -262,7 +245,6 @@ impl<'de> serde::Deserialize<'de> for AggregateExecNode { null_expr: null_expr__.unwrap_or_default(), groups: groups__.unwrap_or_default(), filter_expr: filter_expr__.unwrap_or_default(), - order_by_expr: order_by_expr__.unwrap_or_default(), }) } } diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 8aadc96349ca..3dfd3938615f 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -2193,8 +2193,6 @@ pub struct AggregateExecNode { pub groups: ::prost::alloc::vec::Vec, #[prost(message, repeated, tag = "10")] pub filter_expr: ::prost::alloc::vec::Vec, - #[prost(message, repeated, tag = "11")] - pub order_by_expr: ::prost::alloc::vec::Vec, } #[allow(clippy::derive_partial_eq_without_eq)] #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 359ff67821d8..df01097cfa78 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -1223,12 +1223,6 @@ impl AsExecutionPlan for PhysicalPlanNode { .map(|expr| expr.to_owned().try_into()) .collect::>>()?; - let order_by = exec - .order_by_expr() - .iter() - .map(|expr| expr.to_owned().try_into()) - .collect::>>()?; - let agg = exec .aggr_expr() .iter() @@ -1281,7 +1275,6 @@ impl AsExecutionPlan for PhysicalPlanNode { group_expr_name: group_names, aggr_expr: agg, filter_expr: filter, - order_by_expr: order_by, aggr_expr_name: agg_names, mode: agg_mode as i32, input: Some(Box::new(input)), From 16596b1aa2c19586df09b7baaad5bb97d4d0d72e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 14 Dec 2023 09:19:32 +0300 Subject: [PATCH 48/49] Fix test display --- datafusion/sqllogictest/test_files/groupby.slt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index d1206ace4475..6327e9190a65 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -4299,7 +4299,7 @@ physical_plan SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] --AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 -------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST +------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[ARRAY_AGG(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true @@ -4346,7 +4346,7 @@ physical_plan SortPreservingMergeExec: [a@0 ASC NULLS LAST,b@1 ASC NULLS LAST] --AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----CoalesceBatchesExec: target_batch_size=2 -------SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST +------RepartitionExec: partitioning=Hash([a@0, b@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST,b@1 ASC NULLS LAST --------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(multiple_ordered_table.d), LAST_VALUE(multiple_ordered_table.d), ARRAY_AGG(multiple_ordered_table.d)], ordering_mode=Sorted ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_orderings=[[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true From b378f8026a59fb5851bef2fbbee0070ca2a3b644 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Fri, 15 Dec 2023 18:01:26 +0300 Subject: [PATCH 49/49] Minor --- .../physical-expr/src/aggregate/array_agg_ordered.rs | 10 +--------- datafusion/physical-expr/src/aggregate/first_last.rs | 11 +---------- datafusion/physical-expr/src/aggregate/utils.rs | 10 +++++++++- 3 files changed, 11 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs index 350297b636f0..8965199e0c2c 100644 --- a/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs +++ b/datafusion/physical-expr/src/aggregate/array_agg_ordered.rs @@ -24,7 +24,7 @@ use std::collections::BinaryHeap; use std::fmt::Debug; use std::sync::Arc; -use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; +use crate::aggregate::utils::{down_cast_any_ref, get_sort_options, ordering_fields}; use crate::expressions::format_state_name; use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; @@ -506,14 +506,6 @@ fn merge_ordered_arrays( Ok((merged_values, merged_orderings)) } -/// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { - ordering_req - .iter() - .map(|item| item.options) - .collect::>() -} - #[cfg(test)] mod tests { use crate::aggregate::array_agg_ordered::merge_ordered_arrays; diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 4f5d12f94e66..db7c0c16064f 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -20,7 +20,7 @@ use std::any::Any; use std::sync::Arc; -use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; +use crate::aggregate::utils::{down_cast_any_ref, get_sort_options, ordering_fields}; use crate::expressions::format_state_name; use crate::{ reverse_order_bys, AggregateExpr, LexOrdering, LexOrderingRef, PhysicalExpr, @@ -30,7 +30,6 @@ use crate::{ use arrow::array::{Array, ArrayRef, AsArray, BooleanArray}; use arrow::compute::{self, lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field}; -use arrow_schema::SortOptions; use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::Accumulator; @@ -543,14 +542,6 @@ fn convert_to_sort_cols( .collect::>() } -/// Selects the sort option attribute from all the given `PhysicalSortExpr`s. -fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { - ordering_req - .iter() - .map(|item| item.options) - .collect::>() -} - /// Gets either first, or last value index inside `values` batch according to ordering requirements /// Assumes `values` batch is ordered according to ordering_req already. /// diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index e5421ef5ab7e..7ba7e9d01567 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -26,7 +26,7 @@ use arrow_array::types::{ }; use arrow_array::ArrowNativeTypeOp; use arrow_buffer::ArrowNativeType; -use arrow_schema::{DataType, Field}; +use arrow_schema::{DataType, Field, SortOptions}; use datafusion_common::{exec_err, DataFusionError, Result}; use datafusion_expr::Accumulator; use std::any::Any; @@ -205,3 +205,11 @@ pub(crate) fn ordering_fields( }) .collect() } + +/// Selects the sort option attribute from all the given `PhysicalSortExpr`s. +pub fn get_sort_options(ordering_req: &[PhysicalSortExpr]) -> Vec { + ordering_req + .iter() + .map(|item| item.options) + .collect::>() +}