From 6562c62035e4cc4ada1f7eade8b0dce35e8eda9f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 28 Dec 2022 17:33:33 +0300 Subject: [PATCH 01/19] v1 implementation --- datafusion/core/src/dataframe.rs | 2 + .../core/src/physical_plan/aggregates/hash.rs | 563 ------------------ .../core/src/physical_plan/aggregates/mod.rs | 100 +++- .../physical_plan/aggregates/no_grouping.rs | 179 +++++- .../src/physical_plan/aggregates/row_hash.rs | 477 ++++++++++++--- datafusion/core/tests/sql/window.rs | 322 ++++++++++ .../physical-expr/src/aggregate/count.rs | 5 +- test-utils/src/lib.rs | 5 +- 8 files changed, 954 insertions(+), 699 deletions(-) delete mode 100644 datafusion/core/src/physical_plan/aggregates/hash.rs diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 60d79490cf6fb..51e59cdd1db69 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -834,6 +834,7 @@ mod tests { use arrow::array::Int32Array; use arrow::datatypes::DataType; + use arrow::util::pretty::print_batches; use datafusion_expr::{ avg, cast, count, count_distinct, create_udf, expr, lit, max, min, sum, @@ -945,6 +946,7 @@ mod tests { ]; let df: Vec = df.aggregate(group_expr, aggr_expr)?.collect().await?; + print_batches(&df)?; assert_batches_sorted_eq!( vec![ diff --git a/datafusion/core/src/physical_plan/aggregates/hash.rs b/datafusion/core/src/physical_plan/aggregates/hash.rs deleted file mode 100644 index 64b21ecf9a6b9..0000000000000 --- a/datafusion/core/src/physical_plan/aggregates/hash.rs +++ /dev/null @@ -1,563 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Defines the execution plan for the hash aggregate operation - -use std::collections::VecDeque; -use std::sync::Arc; -use std::task::{Context, Poll}; -use std::vec; - -use ahash::RandomState; -use datafusion_expr::Accumulator; -use futures::stream::BoxStream; -use futures::stream::{Stream, StreamExt}; - -use crate::error::Result; -use crate::execution::context::TaskContext; -use crate::execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; -use crate::physical_plan::aggregates::{ - evaluate_group_by, evaluate_many, AccumulatorItem, AggregateMode, PhysicalGroupBy, -}; -use crate::physical_plan::hash_utils::create_hashes; -use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; -use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; -use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; -use crate::scalar::ScalarValue; - -use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use arrow::{array::ArrayRef, compute, compute::cast}; -use arrow::{ - array::{Array, UInt32Builder}, - error::{ArrowError, Result as ArrowResult}, -}; -use arrow::{ - datatypes::{Schema, SchemaRef}, - record_batch::RecordBatch, -}; -use hashbrown::raw::RawTable; - -/* -The architecture is the following: - -1. An accumulator has state that is updated on each batch. -2. At the end of the aggregation (e.g. end of batches in a partition), the accumulator converts its state to a RecordBatch of a single row -3. The RecordBatches of all accumulators are merged (`concatenate` in `rust/arrow`) together to a single RecordBatch. -4. The state's RecordBatch is `merge`d to a new state -5. The state is mapped to the final value - -Why: - -* Accumulators' state can be statically typed, but it is more efficient to transmit data from the accumulators via `Array` -* The `merge` operation must have access to the state of the aggregators because it uses it to correctly merge -* It uses Arrow's native dynamically typed object, `Array`. -* Arrow shines in batch operations and both `merge` and `concatenate` of uniform types are very performant. - -Example: average - -* the state is `n: u32` and `sum: f64` -* For every batch, we update them accordingly. -* At the end of the accumulation (of a partition), we convert `n` and `sum` to a RecordBatch of 1 row and two columns: `[n, sum]` -* The RecordBatch is (sent back / transmitted over network) -* Once all N record batches arrive, `merge` is performed, which builds a RecordBatch with N rows and 2 columns. -* Finally, `get_value` returns an array with one entry computed from the state -*/ -pub(crate) struct GroupedHashAggregateStream { - stream: BoxStream<'static, ArrowResult>, - schema: SchemaRef, -} - -/// Actual implementation of [`GroupedHashAggregateStream`]. -/// -/// This is wrapped into yet another struct because we need to interact with the async memory management subsystem -/// during poll. To have as little code "weirdness" as possible, we chose to just use [`BoxStream`] together with -/// [`futures::stream::unfold`]. The latter requires a state object, which is [`GroupedHashAggregateStreamV2`]. -struct GroupedHashAggregateStreamInner { - schema: SchemaRef, - input: SendableRecordBatchStream, - mode: AggregateMode, - accumulators: Option, - aggregate_expressions: Vec>>, - - aggr_expr: Vec>, - group_by: PhysicalGroupBy, - - baseline_metrics: BaselineMetrics, - random_state: RandomState, - finished: bool, -} - -impl GroupedHashAggregateStream { - /// Create a new GroupedHashAggregateStream - #[allow(clippy::too_many_arguments)] - pub fn new( - mode: AggregateMode, - schema: SchemaRef, - group_by: PhysicalGroupBy, - aggr_expr: Vec>, - input: SendableRecordBatchStream, - baseline_metrics: BaselineMetrics, - context: Arc, - partition: usize, - ) -> Result { - let timer = baseline_metrics.elapsed_compute().timer(); - - // The expressions to evaluate the batch, one vec of expressions per aggregation. - // Assume create_schema() always put group columns in front of aggr columns, we set - // col_idx_base to group expression count. - let aggregate_expressions = - aggregates::aggregate_expressions(&aggr_expr, &mode, group_by.expr.len())?; - - timer.done(); - - let reservation = - MemoryConsumer::new(format!("GroupedHashAggregateStream[{}]", partition)) - .register(context.memory_pool()); - - let inner = GroupedHashAggregateStreamInner { - schema: Arc::clone(&schema), - mode, - input, - aggr_expr, - group_by, - baseline_metrics, - aggregate_expressions, - accumulators: Some(Accumulators { - reservation, - map: RawTable::with_capacity(0), - group_states: Vec::with_capacity(0), - }), - random_state: Default::default(), - 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 accumulators = - this.accumulators.as_mut().expect("not yet finished"); - let result = group_aggregate_batch( - &this.mode, - &this.random_state, - &this.group_by, - &this.aggr_expr, - batch, - accumulators, - &this.aggregate_expressions, - ); - - 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| { - accumulators.reservation.try_grow(allocated) - }) { - Ok(_) => continue, - Err(e) => Err(ArrowError::ExternalError(Box::new(e))), - } - } - Some(Err(e)) => Err(e), - None => { - this.finished = true; - let timer = this.baseline_metrics.elapsed_compute().timer(); - let result = create_batch_from_map( - &this.mode, - std::mem::take(&mut this.accumulators) - .expect("not yet finished"), - this.group_by.expr.len(), - &this.schema, - ) - .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, stream }) - } -} - -impl Stream for GroupedHashAggregateStream { - type Item = ArrowResult; - - fn poll_next( - mut self: std::pin::Pin<&mut Self>, - cx: &mut Context<'_>, - ) -> Poll> { - let this = &mut *self; - this.stream.poll_next_unpin(cx) - } -} - -impl RecordBatchStream for GroupedHashAggregateStream { - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -/// Perform group-by aggregation for the given [`RecordBatch`]. -/// -/// If successfull, this returns the additional number of bytes that were allocated during this process. -/// -/// TODO: Make this a member function of [`GroupedHashAggregateStream`] -fn group_aggregate_batch( - mode: &AggregateMode, - random_state: &RandomState, - group_by: &PhysicalGroupBy, - aggr_expr: &[Arc], - batch: RecordBatch, - accumulators: &mut Accumulators, - aggregate_expressions: &[Vec>], -) -> Result { - // evaluate the grouping expressions - let group_by_values = evaluate_group_by(group_by, &batch)?; - - // evaluate the aggregation expressions. - // We could evaluate them after the `take`, but since we need to evaluate all - // of them anyways, it is more performant to do it while they are together. - let aggr_input_values = evaluate_many(aggregate_expressions, &batch)?; - - // track memory allocations - let mut allocated = 0usize; - - for grouping_set_values in group_by_values { - // 1.1 construct the key from the group values - // 1.2 construct the mapping key if it does not exist - // 1.3 add the row' index to `indices` - - // track which entries in `accumulators` have rows in this batch to aggregate - let mut groups_with_rows = vec![]; - - // 1.1 Calculate the group keys for the group values - let mut batch_hashes = vec![0; batch.num_rows()]; - create_hashes(&grouping_set_values, random_state, &mut batch_hashes)?; - - for (row, hash) in batch_hashes.into_iter().enumerate() { - let Accumulators { - map, group_states, .. - } = accumulators; - - let entry = map.get_mut(hash, |(_hash, group_idx)| { - // verify that a group that we are inserting with hash is - // actually the same key value as the group in - // existing_idx (aka group_values @ row) - let group_state = &group_states[*group_idx]; - grouping_set_values - .iter() - .zip(group_state.group_by_values.iter()) - .all(|(array, scalar)| scalar.eq_array(array, row)) - }); - - match entry { - // Existing entry for this group value - Some((_hash, group_idx)) => { - let group_state = &mut group_states[*group_idx]; - // 1.3 - if group_state.indices.is_empty() { - groups_with_rows.push(*group_idx); - }; - group_state - .indices - .push_accounted(row as u32, &mut allocated); // remember this row - } - // 1.2 Need to create new entry - None => { - let accumulator_set = aggregates::create_accumulators(aggr_expr)?; - - // Copy group values out of arrays into `ScalarValue`s - let group_by_values = grouping_set_values - .iter() - .map(|col| ScalarValue::try_from_array(col, row)) - .collect::>>()?; - - // Add new entry to group_states and save newly created index - let group_state = GroupState { - group_by_values: group_by_values.into_boxed_slice(), - accumulator_set, - indices: vec![row as u32], // 1.3 - }; - // NOTE: do NOT include the `GroupState` struct size in here because this is captured by - // `group_states` (see allocation down below) - allocated += group_state - .group_by_values - .iter() - .map(|sv| sv.size()) - .sum::() - + (std::mem::size_of::>() - * group_state.accumulator_set.capacity()) - + group_state - .accumulator_set - .iter() - .map(|accu| accu.size()) - .sum::() - + (std::mem::size_of::() * group_state.indices.capacity()); - - let group_idx = group_states.len(); - group_states.push_accounted(group_state, &mut allocated); - groups_with_rows.push(group_idx); - - // for hasher function, use precomputed hash value - map.insert_accounted( - (hash, group_idx), - |(hash, _group_idx)| *hash, - &mut allocated, - ); - } - }; - } - - // Collect all indices + offsets based on keys in this vec - let mut batch_indices: UInt32Builder = UInt32Builder::with_capacity(0); - let mut offsets = vec![0]; - let mut offset_so_far = 0; - for group_idx in groups_with_rows.iter() { - let indices = &accumulators.group_states[*group_idx].indices; - batch_indices.append_slice(indices); - offset_so_far += indices.len(); - offsets.push(offset_so_far); - } - let batch_indices = batch_indices.finish(); - - // `Take` all values based on indices into Arrays - let values: Vec>> = aggr_input_values - .iter() - .map(|array| { - array - .iter() - .map(|array| { - compute::take( - array.as_ref(), - &batch_indices, - None, // None: no index check - ) - .unwrap() - }) - .collect() - // 2.3 - }) - .collect(); - - // 2.1 for each key in this batch - // 2.2 for each aggregation - // 2.3 `slice` from each of its arrays the keys' values - // 2.4 update / merge the accumulator with the values - // 2.5 clear indices - groups_with_rows - .iter() - .zip(offsets.windows(2)) - .try_for_each(|(group_idx, offsets)| { - let group_state = &mut accumulators.group_states[*group_idx]; - // 2.2 - group_state - .accumulator_set - .iter_mut() - .zip(values.iter()) - .map(|(accumulator, aggr_array)| { - ( - accumulator, - aggr_array - .iter() - .map(|array| { - // 2.3 - array.slice(offsets[0], offsets[1] - offsets[0]) - }) - .collect::>(), - ) - }) - .try_for_each(|(accumulator, values)| { - let size_pre = accumulator.size(); - let res = match mode { - AggregateMode::Partial => accumulator.update_batch(&values), - AggregateMode::FinalPartitioned | AggregateMode::Final => { - // note: the aggregation here is over states, not values, thus the merge - accumulator.merge_batch(&values) - } - }; - let size_post = accumulator.size(); - allocated += size_post.saturating_sub(size_pre); - res - }) - // 2.5 - .and({ - group_state.indices.clear(); - Ok(()) - }) - })?; - } - - Ok(allocated) -} - -/// The state that is built for each output group. -#[derive(Debug)] -struct GroupState { - /// The actual group by values, one for each group column - group_by_values: Box<[ScalarValue]>, - - // Accumulator state, one for each aggregate - accumulator_set: Vec, - - /// scratch space used to collect indices for input rows in a - /// bach that have values to aggregate. Reset on each batch - indices: Vec, -} - -/// The state of all the groups -struct Accumulators { - reservation: MemoryReservation, - - /// Logically maps group values to an index in `group_states` - /// - /// Uses the raw API of hashbrown to avoid actually storing the - /// keys in the table - /// - /// keys: u64 hashes of the GroupValue - /// values: (hash, index into `group_states`) - map: RawTable<(u64, usize)>, - - /// State for each group - group_states: Vec, -} - -impl std::fmt::Debug for Accumulators { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - // hashes are not store inline, so could only get values - let map_string = "RawTable"; - f.debug_struct("Accumulators") - .field("map", &map_string) - .field("group_states", &self.group_states) - .finish() - } -} - -/// Create a RecordBatch with all group keys and accumulator' states or values. -/// -/// The output looks like -/// ```text -/// gby_expr1, gby_expr2, ... agg_1, agg2, ... -/// ``` -fn create_batch_from_map( - mode: &AggregateMode, - accumulators: Accumulators, - num_group_expr: usize, - output_schema: &Schema, -) -> ArrowResult { - if accumulators.group_states.is_empty() { - return Ok(RecordBatch::new_empty(Arc::new(output_schema.to_owned()))); - } - let accs = &accumulators.group_states[0].accumulator_set; - let mut acc_data_types: Vec = vec![]; - - // Calculate number/shape of state arrays - match mode { - AggregateMode::Partial => { - for acc in accs.iter() { - let state = acc.state()?; - acc_data_types.push(state.len()); - } - } - AggregateMode::Final | AggregateMode::FinalPartitioned => { - acc_data_types = vec![1; accs.len()]; - } - } - - // make group states mutable - let (mut group_by_values_vec, mut accumulator_set_vec): (Vec<_>, Vec<_>) = - accumulators - .group_states - .into_iter() - .map(|group_state| { - ( - VecDeque::from(group_state.group_by_values.to_vec()), - VecDeque::from(group_state.accumulator_set), - ) - }) - .unzip(); - - // First, output all group by exprs - let mut columns = (0..num_group_expr) - .map(|_| { - ScalarValue::iter_to_array( - group_by_values_vec - .iter_mut() - .map(|x| x.pop_front().expect("invalid group_by_values")), - ) - }) - .collect::>>()?; - - // next, output aggregates: either intermediate state or final output - for (x, &state_len) in acc_data_types.iter().enumerate() { - for y in 0..state_len { - match mode { - AggregateMode::Partial => { - let res = ScalarValue::iter_to_array( - accumulator_set_vec.iter().map(|accumulator_set| { - accumulator_set[x] - .state() - .map(|x| x[y].clone()) - .expect("unexpected accumulator state in hash aggregate") - }), - )?; - - columns.push(res); - } - AggregateMode::Final | AggregateMode::FinalPartitioned => { - let res = ScalarValue::iter_to_array( - accumulator_set_vec.iter_mut().map(|x| { - x.pop_front() - .expect("invalid accumulator_set") - .evaluate() - .unwrap() - }), - )?; - columns.push(res); - } - } - } - } - - // cast output if needed (e.g. for types like Dictionary where - // the intermediate GroupByScalar type was not the same as the - // output - let columns = columns - .iter() - .zip(output_schema.fields().iter()) - .map(|(col, desired_field)| cast(col, desired_field.data_type())) - .collect::>>()?; - - RecordBatch::try_new(Arc::new(output_schema.to_owned()), columns) -} diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 6d7c3c21bc2f4..2d0b7d919beaa 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -18,7 +18,6 @@ //! Aggregates functionalities use crate::execution::context::TaskContext; -use crate::physical_plan::aggregates::hash::GroupedHashAggregateStream; use crate::physical_plan::aggregates::no_grouping::AggregateStream; use crate::physical_plan::metrics::{ BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet, @@ -30,7 +29,7 @@ use crate::physical_plan::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::Result; +use datafusion_common::{Result, ScalarValue}; use datafusion_expr::Accumulator; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ @@ -39,19 +38,22 @@ use datafusion_physical_expr::{ use std::any::Any; use std::collections::HashMap; +use parquet::schema::printer::print_schema; use std::sync::Arc; -mod hash; mod no_grouping; mod row_hash; -use crate::physical_plan::aggregates::row_hash::GroupedHashAggregateStreamV2; +use crate::physical_plan::aggregates::row_hash::{ + read_as_batch, GroupedHashAggregateStreamV2, RowAggregationState, +}; use crate::physical_plan::EquivalenceProperties; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::row_accumulator::RowAccumulator; use datafusion_physical_expr::equivalence::project_equivalence_properties; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::normalize_out_expr_with_alias_schema; +use datafusion_row::accessor::RowAccessor; use datafusion_row::{row_supported, RowType}; /// Hash aggregate modes @@ -153,7 +155,6 @@ impl PhysicalGroupBy { enum StreamType { AggregateStream(AggregateStream), GroupedHashAggregateStreamV2(GroupedHashAggregateStreamV2), - GroupedHashAggregateStream(GroupedHashAggregateStream), } impl From for SendableRecordBatchStream { @@ -161,7 +162,6 @@ impl From for SendableRecordBatchStream { match stream { StreamType::AggregateStream(stream) => Box::pin(stream), StreamType::GroupedHashAggregateStreamV2(stream) => Box::pin(stream), - StreamType::GroupedHashAggregateStream(stream) => Box::pin(stream), } } } @@ -285,9 +285,8 @@ impl AggregateExec { ) -> Result { let batch_size = context.session_config().batch_size(); let input = self.input.execute(partition, Arc::clone(&context))?; - + const VERSION: usize = 1; let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); - if self.group_by.expr.is_empty() { Ok(StreamType::AggregateStream(AggregateStream::new( self.mode, @@ -298,7 +297,7 @@ impl AggregateExec { context, partition, )?)) - } else if self.row_aggregate_supported() { + } else { Ok(StreamType::GroupedHashAggregateStreamV2( GroupedHashAggregateStreamV2::new( self.mode, @@ -312,19 +311,6 @@ impl AggregateExec { partition, )?, )) - } else { - Ok(StreamType::GroupedHashAggregateStream( - GroupedHashAggregateStream::new( - self.mode, - self.schema.clone(), - self.group_by.clone(), - self.aggr_expr.clone(), - input, - baseline_metrics, - context, - partition, - )?, - )) } } } @@ -634,9 +620,14 @@ fn create_accumulators_v2( /// final value (mode = Final) or states (mode = Partial) fn finalize_aggregation( accumulators: &[AccumulatorItem], + row_accumulators: &[AccumulatorItemV2], mode: &AggregateMode, + aggr_schema: &Schema, + aggr_state: &mut RowAggregationState, + output_schema: &Schema, + indices: &Vec>, ) -> datafusion_common::Result> { - match mode { + let mut acc_res = match mode { AggregateMode::Partial => { // build the vector of states let a = accumulators @@ -657,7 +648,65 @@ fn finalize_aggregation( .map(|accumulator| accumulator.evaluate().map(|v| v.to_array())) .collect::>>() } + }?; + + let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); + + let mut state_buffers = vec![aggr_state.group_states[0].aggregation_buffer.clone()]; + let mut columns: Vec = vec![]; + match mode { + AggregateMode::Partial => columns.extend(read_as_batch( + &state_buffers, + aggr_schema, + RowType::WordAligned, + )), + AggregateMode::Final | AggregateMode::FinalPartitioned => { + let mut results: Vec> = vec![vec![]; row_accumulators.len()]; + for buffer in state_buffers.iter_mut() { + state_accessor.point_to(0, buffer); + for (i, acc) in row_accumulators.iter().enumerate() { + results[i].push(acc.evaluate(&state_accessor).unwrap()); + } + } + // We skip over the first `columns.len()` elements. + // + // This shouldn't panic if the `output_schema` has enough fields. + let remaining_field_iterator = output_schema.fields()[columns.len()..].iter(); + + for (scalars, field) in results.into_iter().zip(remaining_field_iterator) { + if !scalars.is_empty() { + columns.push(ScalarValue::iter_to_array(scalars)?); + } else { + columns.push(arrow::array::new_empty_array(field.data_type())) + } + } + } + }; + let empty_arr = ScalarValue::iter_to_array(vec![ScalarValue::Null])?; + let n_res = indices[0] + .iter() + .map(|(_, range)| range.1 - range.0) + .sum::() + + indices[1] + .iter() + .map(|(_, range)| range.1 - range.0) + .sum::(); + let mut res = vec![empty_arr; n_res]; + let results = vec![acc_res, columns]; + for outer in [0, 1] { + let mut start_idx = 0; + let cur_res = &results[outer]; + let cur_indices = &indices[outer]; + // println!("cur res:{:?}", cur_res); + // println!("cur_indices:{:?}", cur_indices); + for (_idx, range) in cur_indices.iter() { + for res_idx in range.0..range.1 { + res[res_idx] = cur_res[start_idx].clone(); + start_idx += 1; + } + } } + Ok(res) } /// Evaluates expressions against a record batch. @@ -1183,7 +1232,10 @@ mod tests { assert!(matches!(stream, StreamType::AggregateStream(_))); } 1 => { - assert!(matches!(stream, StreamType::GroupedHashAggregateStream(_))); + assert!(matches!( + stream, + StreamType::GroupedHashAggregateStreamV2(_) + )); } 2 => { assert!(matches!( diff --git a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs index 8a312abafd9ba..f00d79652b1a9 100644 --- a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs +++ b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs @@ -19,11 +19,13 @@ use crate::execution::context::TaskContext; use crate::physical_plan::aggregates::{ - aggregate_expressions, create_accumulators, finalize_aggregation, AccumulatorItem, - AggregateMode, + aggregate_expressions, create_accumulators, create_accumulators_v2, evaluate_many, + finalize_aggregation, AccumulatorItem, AccumulatorItemV2, AggregateMode, }; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; +use arrow::array::{Array, ArrayRef}; +use arrow::compute; use arrow::datatypes::SchemaRef; use arrow::error::{ArrowError, Result as ArrowResult}; use arrow::record_batch::RecordBatch; @@ -34,7 +36,14 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use crate::physical_plan::aggregates::row_hash::{ + aggr_state_schema, RowAggregationState, RowGroupState, +}; +use datafusion_row::accessor::RowAccessor; +use datafusion_row::layout::RowLayout; +use datafusion_row::RowType; use futures::stream::{Stream, StreamExt}; +use hashbrown::raw::RawTable; /// stream struct for aggregation without grouping columns pub(crate) struct AggregateStream { @@ -53,9 +62,15 @@ struct AggregateStreamInner { input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, aggregate_expressions: Vec>>, + row_aggregate_expressions: Vec>>, accumulators: Vec, + row_accumulators: Vec, reservation: MemoryReservation, finished: bool, + row_aggr_state: RowAggregationState, + row_aggr_layout: Arc, + row_aggr_schema: SchemaRef, + indices: Vec>, } impl AggregateStream { @@ -69,8 +84,76 @@ impl AggregateStream { context: Arc, partition: usize, ) -> Result { - let aggregate_expressions = aggregate_expressions(&aggr_expr, &mode, 0)?; - let accumulators = create_accumulators(&aggr_expr)?; + let mut row_agg_indices = vec![]; + let mut normal_agg_indices = vec![]; + let mut start_idx = 0; + for idx in 0..aggr_expr.len() { + let n_field = match mode { + AggregateMode::Partial => aggr_expr[idx].state_fields()?.len(), + _ => 1, + }; + // println!("aggr_expr[idx]: {:?}", aggr_expr[idx]); + // println!("n_field:{:?}", n_field); + if aggr_expr[idx].row_accumulator_supported() { + row_agg_indices.push((idx, (start_idx, start_idx + n_field))); + } else { + normal_agg_indices.push((idx, (start_idx, start_idx + n_field))); + } + start_idx += n_field; + } + let indices = vec![normal_agg_indices, row_agg_indices]; + println!("indices:{:?}", indices); + let row_aggr_exprs = aggr_expr + .clone() + .into_iter() + .filter(|elem| elem.row_accumulator_supported()) + .collect::>(); + let normal_aggr_exprs = aggr_expr + .clone() + .into_iter() + .filter(|elem| !elem.row_accumulator_supported()) + .collect::>(); + + let row_aggr_schema = aggr_state_schema(&row_aggr_exprs)?; + + let row_aggr_layout = + Arc::new(RowLayout::new(&row_aggr_schema, RowType::WordAligned)); + let reservation = + MemoryConsumer::new(format!("GroupedHashAggregateStreamV2[{}]", partition)) + .register(context.memory_pool()); + + // Add new entry to group_states and save newly created index + let group_state = RowGroupState { + group_by_values: Box::new([]), + aggregation_buffer: vec![0; row_aggr_layout.fixed_part_width()], + accumulator_set: vec![], + indices: vec![0 as u32], // 1.3 + }; + let row_aggr_state = RowAggregationState { + reservation, + map: RawTable::with_capacity(0), + group_states: vec![group_state], + }; + + let all_aggregate_expressions = aggregate_expressions(&aggr_expr, &mode, 0)?; + let mut normal_aggregate_expressions = vec![]; + for (idx, _) in &indices[0] { + normal_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) + } + let mut row_aggregate_expressions = vec![]; + for (idx, _) in &indices[1] { + row_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) + } + // let normal_aggregate_expressions = aggregate_expressions(&normal_aggr_exprs, &mode, 0)?; + // let row_aggregate_expressions = aggregate_expressions(&row_aggr_exprs, &mode, 0)?; + // println!("all_aggregate_expressions:{:?}", all_aggregate_expressions); + // println!( + // "normal_aggregate_expressions:{:?}", + // normal_aggregate_expressions + // ); + // println!("row_aggregate_expressions:{:?}", row_aggregate_expressions); + let normal_accumulators = create_accumulators(&normal_aggr_exprs)?; + let row_accumulators = create_accumulators_v2(&row_aggr_exprs)?; let reservation = MemoryConsumer::new(format!("AggregateStream[{}]", partition)) .register(context.memory_pool()); @@ -80,10 +163,16 @@ impl AggregateStream { mode, input, baseline_metrics, - aggregate_expressions, - accumulators, + aggregate_expressions: normal_aggregate_expressions, + row_aggregate_expressions, + accumulators: normal_accumulators, + row_accumulators, reservation, finished: false, + row_aggr_state, + row_aggr_layout, + row_aggr_schema, + indices, }; let stream = futures::stream::unfold(inner, |mut this| async move { if this.finished { @@ -100,7 +189,11 @@ impl AggregateStream { &this.mode, &batch, &mut this.accumulators, + &mut this.row_accumulators, &this.aggregate_expressions, + &this.row_aggregate_expressions, + &mut this.row_aggr_state, + this.row_aggr_layout.clone(), ); timer.done(); @@ -119,12 +212,20 @@ impl AggregateStream { None => { this.finished = true; let timer = this.baseline_metrics.elapsed_compute().timer(); - let result = finalize_aggregation(&this.accumulators, &this.mode) - .map_err(|e| ArrowError::ExternalError(Box::new(e))) - .and_then(|columns| { - RecordBatch::try_new(this.schema.clone(), columns) - }) - .record_output(&this.baseline_metrics); + let result = finalize_aggregation( + &this.accumulators, + &this.row_accumulators, + &this.mode, + &this.row_aggr_schema, + &mut this.row_aggr_state, + &this.schema, + &this.indices, + ) + .map_err(|e| ArrowError::ExternalError(Box::new(e))) + .and_then(|columns| { + RecordBatch::try_new(this.schema.clone(), columns) + }) + .record_output(&this.baseline_metrics); timer.done(); @@ -172,8 +273,60 @@ fn aggregate_batch( mode: &AggregateMode, batch: &RecordBatch, accumulators: &mut [AccumulatorItem], + row_accumulators: &mut [AccumulatorItemV2], expressions: &[Vec>], + row_expressions: &[Vec>], + row_aggr_state: &mut RowAggregationState, + state_layout: Arc, ) -> Result { + let RowAggregationState { + map, group_states, .. + } = row_aggr_state; + let group_state = &mut group_states[0]; + let aggr_input_values = evaluate_many(row_expressions, &batch)?; + // println!("aggr_input_values:{:?}", aggr_input_values); + // // `Take` all values based on indices into Arrays + // let values: Vec>> = aggr_input_values + // .iter() + // .map(|array| { + // array + // .iter() + // .map(|array| { + // compute::take( + // array.as_ref(), + // &batch_indices, + // None, // None: no index check + // ) + // .unwrap() + // }) + // .collect() + // // 2.3 + // }) + // .collect(); + row_accumulators + .iter_mut() + .zip(aggr_input_values.iter()) + .map(|(accumulator, aggr_array)| (accumulator, aggr_array)) + .try_for_each(|(accumulator, values)| { + let mut state_accessor = RowAccessor::new_from_layout(state_layout.clone()); + state_accessor.point_to(0, group_state.aggregation_buffer.as_mut_slice()); + // println!("row acc values:{:?}, mode:{:?}", values, mode); + match mode { + AggregateMode::Partial => { + accumulator.update_batch(&values, &mut state_accessor) + } + AggregateMode::FinalPartitioned | AggregateMode::Final => { + // note: the aggregation here is over states, not values, thus the merge + accumulator.merge_batch(&values, &mut state_accessor) + } + } + }) + // 2.5 + .and({ + group_state.indices.clear(); + Ok(()) + })?; + let mut allocated = 0usize; // 1.1 iterate accumulators and respective expressions together @@ -181,6 +334,8 @@ fn aggregate_batch( // 1.3 update / merge accumulators with the expressions' values // 1.1 + // println!("expressions:{:?}", expressions); + // println!("row_expressions:{:?}", row_expressions); accumulators .iter_mut() .zip(expressions) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index e769397871efe..78c1f731b0ed1 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -17,6 +17,7 @@ //! Hash aggregation through row format +use std::collections::VecDeque; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; @@ -29,8 +30,8 @@ use crate::error::Result; use crate::execution::context::TaskContext; use crate::execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use crate::physical_plan::aggregates::{ - evaluate_group_by, evaluate_many, group_schema, AccumulatorItemV2, AggregateMode, - PhysicalGroupBy, + evaluate_group_by, evaluate_many, group_schema, AccumulatorItem, AccumulatorItemV2, + AggregateMode, PhysicalGroupBy, }; use crate::physical_plan::hash_utils::create_row_hashes; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; @@ -38,6 +39,7 @@ use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use crate::physical_plan::common::transpose; use arrow::compute::cast; use arrow::datatypes::Schema; use arrow::{array::ArrayRef, compute}; @@ -46,12 +48,14 @@ use arrow::{ error::{ArrowError, Result as ArrowResult}, }; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; -use datafusion_common::ScalarValue; +use datafusion_common::{DataFusionError, ScalarValue}; +use datafusion_expr::Accumulator; +use datafusion_physical_expr::hash_utils::create_hashes; use datafusion_row::accessor::RowAccessor; use datafusion_row::layout::RowLayout; use datafusion_row::reader::{read_row, RowReader}; use datafusion_row::writer::{write_row, RowWriter}; -use datafusion_row::{MutableRecordBatch, RowType}; +use datafusion_row::{row_supported, MutableRecordBatch, RowType}; use hashbrown::raw::RawTable; /// Grouping aggregate with row-format aggregation states inside. @@ -84,15 +88,18 @@ struct GroupedHashAggregateStreamV2Inner { schema: SchemaRef, input: SendableRecordBatchStream, mode: AggregateMode, - aggr_state: AggregationState, - aggregate_expressions: Vec>>, + normal_aggr_expr: Vec>, + row_aggr_state: RowAggregationState, + normal_aggregate_expressions: Vec>>, + row_aggregate_expressions: Vec>>, group_by: PhysicalGroupBy, - accumulators: Vec, + normal_accumulators: Vec, + row_accumulators: Vec, group_schema: SchemaRef, - aggr_schema: SchemaRef, - aggr_layout: Arc, + row_aggr_schema: SchemaRef, + row_aggr_layout: Arc, baseline_metrics: BaselineMetrics, random_state: RandomState, @@ -101,9 +108,10 @@ struct GroupedHashAggregateStreamV2Inner { /// if the result is chunked into batches, /// last offset is preserved for continuation. row_group_skip_position: usize, + indices: Vec>, } -fn aggr_state_schema(aggr_expr: &[Arc]) -> Result { +pub fn aggr_state_schema(aggr_expr: &[Arc]) -> Result { let fields = aggr_expr .iter() .flat_map(|expr| expr.state_fields().unwrap().into_iter()) @@ -111,6 +119,10 @@ fn aggr_state_schema(aggr_expr: &[Arc]) -> Result Ok(Arc::new(Schema::new(fields))) } +fn is_supported(elem: &Arc, group_schema: &Schema) -> bool { + elem.row_accumulator_supported() && row_supported(group_schema, RowType::Compact) +} + impl GroupedHashAggregateStreamV2 { /// Create a new GroupedRowHashAggregateStream #[allow(clippy::too_many_arguments)] @@ -127,24 +139,62 @@ impl GroupedHashAggregateStreamV2 { ) -> Result { let timer = baseline_metrics.elapsed_compute().timer(); + let group_schema = group_schema(&schema, group_by.expr.len()); + let mut row_agg_indices = vec![]; + let mut normal_agg_indices = vec![]; + let mut start_idx = group_by.expr.len(); + for idx in 0..aggr_expr.len() { + let n_field = match mode { + AggregateMode::Partial => aggr_expr[idx].state_fields()?.len(), + _ => 1, + }; + if is_supported(&aggr_expr[idx], &group_schema) { + row_agg_indices.push((idx, (start_idx, start_idx + n_field))); + } else { + normal_agg_indices.push((idx, (start_idx, start_idx + n_field))); + } + start_idx += n_field; + } + let indices = vec![normal_agg_indices, row_agg_indices]; + + let row_aggr_exprs = aggr_expr + .clone() + .into_iter() + .filter(|elem| is_supported(elem, &group_schema)) + .collect::>(); + let normal_aggr_exprs = aggr_expr + .clone() + .into_iter() + .filter(|elem| !is_supported(elem, &group_schema)) + .collect::>(); // The expressions to evaluate the batch, one vec of expressions per aggregation. // Assume create_schema() always put group columns in front of aggr columns, we set // col_idx_base to group expression count. - let aggregate_expressions = + let all_aggregate_expressions = aggregates::aggregate_expressions(&aggr_expr, &mode, group_by.expr.len())?; + let mut normal_aggregate_expressions = vec![]; + for (idx, _) in &indices[0] { + normal_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) + } + let mut row_aggregate_expressions = vec![]; + for (idx, _) in &indices[1] { + row_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) + } - let accumulators = aggregates::create_accumulators_v2(&aggr_expr)?; + // let accumulators = aggregates::create_accumulators_v2(&aggr_expr)?; + let normal_accumulators = aggregates::create_accumulators(&normal_aggr_exprs)?; + let row_accumulators = aggregates::create_accumulators_v2(&row_aggr_exprs)?; - let group_schema = group_schema(&schema, group_by.expr.len()); - let aggr_schema = aggr_state_schema(&aggr_expr)?; + let row_aggr_schema = aggr_state_schema(&row_aggr_exprs)?; - let aggr_layout = Arc::new(RowLayout::new(&aggr_schema, RowType::WordAligned)); - let reservation = + let row_aggr_layout = + Arc::new(RowLayout::new(&row_aggr_schema, RowType::WordAligned)); + let row_reservation = MemoryConsumer::new(format!("GroupedHashAggregateStreamV2[{}]", partition)) .register(context.memory_pool()); - let aggr_state = AggregationState { - reservation, + let row_aggr_state = RowAggregationState { + reservation: row_reservation, map: RawTable::with_capacity(0), group_states: Vec::with_capacity(0), }; @@ -156,16 +206,20 @@ impl GroupedHashAggregateStreamV2 { mode, input, group_by, - accumulators, + normal_aggr_expr: normal_aggr_exprs, + normal_accumulators, + row_accumulators, group_schema, - aggr_schema, - aggr_layout, + row_aggr_schema, + row_aggr_layout, baseline_metrics, - aggregate_expressions, - aggr_state, + normal_aggregate_expressions, + row_aggregate_expressions, + row_aggr_state, random_state: Default::default(), batch_size, row_group_skip_position: 0, + indices, }; let stream = futures::stream::unfold(inner, |mut this| async move { @@ -180,12 +234,14 @@ impl GroupedHashAggregateStreamV2 { &this.mode, &this.random_state, &this.group_by, - &mut this.accumulators, + &this.normal_aggr_expr, + &mut this.row_accumulators, &this.group_schema, - this.aggr_layout.clone(), + this.row_aggr_layout.clone(), batch, - &mut this.aggr_state, - &this.aggregate_expressions, + &mut this.row_aggr_state, + &this.normal_aggregate_expressions, + &this.row_aggregate_expressions, ); timer.done(); @@ -194,7 +250,7 @@ impl GroupedHashAggregateStreamV2 { // 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.aggr_state.reservation.try_grow(allocated) + this.row_aggr_state.reservation.try_grow(allocated) }) { Ok(_) => continue, Err(e) => Err(ArrowError::ExternalError(Box::new(e))), @@ -206,12 +262,14 @@ impl GroupedHashAggregateStreamV2 { let result = create_batch_from_map( &this.mode, &this.group_schema, - &this.aggr_schema, + &this.row_aggr_schema, this.batch_size, this.row_group_skip_position, - &mut this.aggr_state, - &mut this.accumulators, + &mut this.row_aggr_state, + &mut this.row_accumulators, &this.schema, + &this.indices, + this.group_by.expr.len(), ); timer.done(); @@ -269,29 +327,33 @@ fn group_aggregate_batch( mode: &AggregateMode, random_state: &RandomState, grouping_set: &PhysicalGroupBy, - accumulators: &mut [AccumulatorItemV2], + normal_aggr_expr: &[Arc], + row_accumulators: &mut [AccumulatorItemV2], group_schema: &Schema, state_layout: Arc, batch: RecordBatch, - aggr_state: &mut AggregationState, - aggregate_expressions: &[Vec>], + aggr_state: &mut RowAggregationState, + normal_aggregate_expressions: &[Vec>], + row_aggregate_expressions: &[Vec>], ) -> Result { // evaluate the grouping expressions - let grouping_by_values = evaluate_group_by(grouping_set, &batch)?; - - let AggregationState { - map, group_states, .. + let group_by_values = evaluate_group_by(grouping_set, &batch)?; + let mut row_allocated = 0usize; + // track memory allocations + let mut normal_allocated = 0usize; + let RowAggregationState { + map: row_map, + group_states: row_group_states, + .. } = aggr_state; - let mut allocated = 0usize; - - for group_values in grouping_by_values { - let group_rows: Vec> = create_group_rows(group_values, group_schema); - // evaluate the aggregation expressions. - // We could evaluate them after the `take`, but since we need to evaluate all - // of them anyways, it is more performant to do it while they are together. - let aggr_input_values = evaluate_many(aggregate_expressions, &batch)?; + // evaluate the aggregation expressions. + // We could evaluate them after the `take`, but since we need to evaluate all + // of them anyways, it is more performant to do it while they are together. + let row_aggr_input_values = evaluate_many(row_aggregate_expressions, &batch)?; + let normal_aggr_input_values = evaluate_many(normal_aggregate_expressions, &batch)?; + for group_values in &group_by_values { // 1.1 construct the key from the group values // 1.2 construct the mapping key if it does not exist // 1.3 add the row' index to `indices` @@ -301,21 +363,25 @@ fn group_aggregate_batch( // 1.1 Calculate the group keys for the group values let mut batch_hashes = vec![0; batch.num_rows()]; - create_row_hashes(&group_rows, random_state, &mut batch_hashes)?; + create_hashes(&group_values, random_state, &mut batch_hashes)?; for (row, hash) in batch_hashes.into_iter().enumerate() { - let entry = map.get_mut(hash, |(_hash, group_idx)| { + let entry = row_map.get_mut(hash, |(_hash, group_idx)| { // verify that a group that we are inserting with hash is // actually the same key value as the group in // existing_idx (aka group_values @ row) - let group_state = &group_states[*group_idx]; - group_rows[row] == group_state.group_by_values + let group_state = &row_group_states[*group_idx]; + let group_by_values = get_at_row(group_values, row).unwrap(); + group_by_values + .iter() + .zip(group_state.group_by_values.iter()) + .all(|(lhs, rhs)| lhs.eq(rhs)) }); match entry { // Existing entry for this group value Some((_hash, group_idx)) => { - let group_state = &mut group_states[*group_idx]; + let group_state = &mut row_group_states[*group_idx]; // 1.3 if group_state.indices.is_empty() { @@ -324,34 +390,49 @@ fn group_aggregate_batch( group_state .indices - .push_accounted(row as u32, &mut allocated); // remember this row + .push_accounted(row as u32, &mut row_allocated); // remember this row } // 1.2 Need to create new entry None => { + let group_by_values = get_at_row(group_values, row).unwrap(); + let accumulator_set = + aggregates::create_accumulators(normal_aggr_expr)?; // Add new entry to group_states and save newly created index let group_state = RowGroupState { - group_by_values: group_rows[row].clone(), + group_by_values: group_by_values.into_boxed_slice(), aggregation_buffer: vec![0; state_layout.fixed_part_width()], + accumulator_set, indices: vec![row as u32], // 1.3 }; - let group_idx = group_states.len(); + let group_idx = row_group_states.len(); // NOTE: do NOT include the `RowGroupState` struct size in here because this is captured by // `group_states` (see allocation down below) - allocated += (std::mem::size_of::() - * group_state.group_by_values.capacity()) + row_allocated += group_state + .group_by_values + .iter() + .map(|sv| sv.size()) + .sum::() + (std::mem::size_of::() * group_state.aggregation_buffer.capacity()) + (std::mem::size_of::() * group_state.indices.capacity()); + normal_allocated += (std::mem::size_of::>() + * group_state.accumulator_set.capacity()) + + group_state + .accumulator_set + .iter() + .map(|accu| accu.size()) + .sum::(); + // for hasher function, use precomputed hash value - map.insert_accounted( + row_map.insert_accounted( (hash, group_idx), |(hash, _group_index)| *hash, - &mut allocated, + &mut row_allocated, ); - group_states.push_accounted(group_state, &mut allocated); + row_group_states.push_accounted(group_state, &mut row_allocated); groups_with_rows.push(group_idx); } @@ -363,7 +444,7 @@ fn group_aggregate_batch( let mut offsets = vec![0]; let mut offset_so_far = 0; for group_idx in groups_with_rows.iter() { - let indices = &group_states[*group_idx].indices; + let indices = &row_group_states[*group_idx].indices; batch_indices.append_slice(indices); offset_so_far += indices.len(); offsets.push(offset_so_far); @@ -371,7 +452,7 @@ fn group_aggregate_batch( let batch_indices = batch_indices.finish(); // `Take` all values based on indices into Arrays - let values: Vec>> = aggr_input_values + let row_values: Vec>> = row_aggr_input_values .iter() .map(|array| { array @@ -389,6 +470,24 @@ fn group_aggregate_batch( }) .collect(); + // `Take` all values based on indices into Arrays + let normal_values: Vec>> = normal_aggr_input_values + .iter() + .map(|array| { + array + .iter() + .map(|array| { + compute::take( + array.as_ref(), + &batch_indices, + None, // None: no index check + ) + .unwrap() + }) + .collect() + // 2.3 + }) + .collect(); // 2.1 for each key in this batch // 2.2 for each aggregation // 2.3 `slice` from each of its arrays the keys' values @@ -398,11 +497,11 @@ fn group_aggregate_batch( .iter() .zip(offsets.windows(2)) .try_for_each(|(group_idx, offsets)| { - let group_state = &mut group_states[*group_idx]; + let group_state = &mut row_group_states[*group_idx]; // 2.2 - accumulators + row_accumulators .iter_mut() - .zip(values.iter()) + .zip(row_values.iter()) .map(|(accumulator, aggr_array)| { ( accumulator, @@ -431,33 +530,70 @@ fn group_aggregate_batch( } }) // 2.5 + .and({ Ok(()) })?; + // normal accumulators + group_state + .accumulator_set + .iter_mut() + .zip(normal_values.iter()) + .map(|(accumulator, aggr_array)| { + ( + accumulator, + aggr_array + .iter() + .map(|array| { + // 2.3 + array.slice(offsets[0], offsets[1] - offsets[0]) + }) + .collect::>(), + ) + }) + .try_for_each(|(accumulator, values)| { + let size_pre = accumulator.size(); + let res = match mode { + AggregateMode::Partial => accumulator.update_batch(&values), + AggregateMode::FinalPartitioned | AggregateMode::Final => { + // note: the aggregation here is over states, not values, thus the merge + accumulator.merge_batch(&values) + } + }; + let size_post = accumulator.size(); + normal_allocated += size_post.saturating_sub(size_pre); + res + }) + // 2.5 .and({ group_state.indices.clear(); Ok(()) - }) + })?; + + Ok::<(), DataFusionError>(()) })?; } - Ok(allocated) + Ok(row_allocated + normal_allocated) } /// The state that is built for each output group. #[derive(Debug)] -struct RowGroupState { +pub struct RowGroupState { /// The actual group by values, stored sequentially - group_by_values: Vec, + pub group_by_values: Box<[ScalarValue]>, // Accumulator state, stored sequentially - aggregation_buffer: Vec, + pub aggregation_buffer: Vec, + + // Accumulator state, one for each aggregate + pub accumulator_set: Vec, /// scratch space used to collect indices for input rows in a /// bach that have values to aggregate. Reset on each batch - indices: Vec, + pub indices: Vec, } /// The state of all the groups -struct AggregationState { - reservation: MemoryReservation, +pub struct RowAggregationState { + pub reservation: MemoryReservation, /// Logically maps group values to an index in `group_states` /// @@ -466,13 +602,13 @@ struct AggregationState { /// /// keys: u64 hashes of the GroupValue /// values: (hash, index into `group_states`) - map: RawTable<(u64, usize)>, + pub map: RawTable<(u64, usize)>, /// State for each group - group_states: Vec, + pub group_states: Vec, } -impl std::fmt::Debug for AggregationState { +impl std::fmt::Debug for RowAggregationState { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { // hashes are not store inline, so could only get values let map_string = "RawTable"; @@ -503,57 +639,78 @@ fn create_batch_from_map( aggr_schema: &Schema, batch_size: usize, skip_items: usize, - aggr_state: &mut AggregationState, - accumulators: &mut [AccumulatorItemV2], + row_aggr_state: &mut RowAggregationState, + row_accumulators: &mut [AccumulatorItemV2], output_schema: &Schema, + indices: &Vec>, + num_group_expr: usize, ) -> ArrowResult> { - if skip_items > aggr_state.group_states.len() { + if skip_items > row_aggr_state.group_states.len() { return Ok(None); } + if row_aggr_state.group_states.is_empty() { + return Ok(Some(RecordBatch::new_empty(Arc::new( + output_schema.to_owned(), + )))); + } - if aggr_state.group_states.is_empty() { + let group_buffers = row_aggr_state + .group_states + .iter() + .skip(skip_items) + .take(batch_size) + .map(|gs| (gs.group_by_values.clone())) + .collect::>(); + + // let mut group_by_columns = read_as_batch(&group_buffers, group_schema, RowType::Compact)?; + let n_row = group_buffers.len(); + if n_row == 0 { return Ok(Some(RecordBatch::new_empty(Arc::new( output_schema.to_owned(), )))); } + // First, output all group by exprs + let mut group_by_columns = (0..num_group_expr) + .map(|idx| { + ScalarValue::iter_to_array(group_buffers.iter().map(|x| x[idx].clone())) + }) + .collect::>>()?; + let mut row_columns = vec![]; let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); - - let (group_buffers, mut state_buffers): (Vec<_>, Vec<_>) = aggr_state + let mut state_buffers = row_aggr_state .group_states .iter() .skip(skip_items) .take(batch_size) - .map(|gs| (gs.group_by_values.clone(), gs.aggregation_buffer.clone())) - .unzip(); - - let mut columns: Vec = - read_as_batch(&group_buffers, group_schema, RowType::Compact); + .map(|gs| gs.aggregation_buffer.clone()) + .collect::>(); match mode { - AggregateMode::Partial => columns.extend(read_as_batch( + AggregateMode::Partial => row_columns.extend(read_as_batch( &state_buffers, aggr_schema, RowType::WordAligned, )), AggregateMode::Final | AggregateMode::FinalPartitioned => { - let mut results: Vec> = vec![vec![]; accumulators.len()]; + let mut results: Vec> = vec![vec![]; row_accumulators.len()]; for buffer in state_buffers.iter_mut() { state_accessor.point_to(0, buffer); - for (i, acc) in accumulators.iter().enumerate() { + for (i, acc) in row_accumulators.iter().enumerate() { results[i].push(acc.evaluate(&state_accessor).unwrap()); } } // We skip over the first `columns.len()` elements. // // This shouldn't panic if the `output_schema` has enough fields. - let remaining_field_iterator = output_schema.fields()[columns.len()..].iter(); + let remaining_field_iterator = + output_schema.fields()[group_by_columns.len()..].iter(); for (scalars, field) in results.into_iter().zip(remaining_field_iterator) { if !scalars.is_empty() { - columns.push(ScalarValue::iter_to_array(scalars)?); + row_columns.push(ScalarValue::iter_to_array(scalars)?); } else { - columns.push(arrow::array::new_empty_array(field.data_type())) + row_columns.push(arrow::array::new_empty_array(field.data_type())) } } } @@ -562,16 +719,132 @@ fn create_batch_from_map( // cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output - let columns = columns + let row_column_indices = indices[1].clone(); + let mut start_idx = 0; + let mut new_row_columns = vec![]; + for (_row_column_idx, range) in row_column_indices.iter() { + for idx in range.0..range.1 { + let desired_datatype = output_schema.fields()[idx].data_type(); + new_row_columns.push(cast(&row_columns[start_idx], desired_datatype)?); + start_idx += 1; + } + } + + let row_columns = new_row_columns; + // RecordBatch::try_new(Arc::new(output_schema.to_owned()), row_columns).map(Some) + + let mut columns = vec![]; + let accs = &row_aggr_state.group_states[0].accumulator_set; + let mut acc_data_types: Vec = vec![]; + + // Calculate number/shape of state arrays + match mode { + AggregateMode::Partial => { + for acc in accs.iter() { + let state = acc.state()?; + acc_data_types.push(state.len()); + } + } + AggregateMode::Final | AggregateMode::FinalPartitioned => { + acc_data_types = vec![1; accs.len()]; + } + } + + // make group states mutable + let mut accumulator_set_vec: Vec<_> = row_aggr_state + .group_states + .iter() + .map(|group_state| &group_state.accumulator_set) + .collect(); + + // next, output aggregates: either intermediate state or final output + let mut start_idx = 0; + for (x, &state_len) in acc_data_types.iter().enumerate() { + for y in 0..state_len { + match mode { + AggregateMode::Partial => { + let res = ScalarValue::iter_to_array( + accumulator_set_vec.iter().map(|accumulator_set| { + accumulator_set[x] + .state() + .map(|x| x[y].clone()) + .expect("unexpected accumulator state in hash aggregate") + }), + )?; + + columns.push(res); + } + AggregateMode::Final | AggregateMode::FinalPartitioned => { + let res = ScalarValue::iter_to_array( + accumulator_set_vec.iter_mut().map(|accumulator_set| { + let res = accumulator_set[x] + .evaluate() + .expect("unexpected accumulator state in hash aggregate"); + start_idx += 1; + res + }), + )?; + columns.push(res); + } + } + } + } + // cast output if needed (e.g. for types like Dictionary where + // the intermediate GroupByScalar type was not the same as the + // output + + let column_indices = indices[0].clone(); + let mut start_idx = 0; + let mut new_columns = vec![]; + for (_column_idx, range) in column_indices.iter() { + for idx in range.0..range.1 { + let desired_datatype = output_schema.fields()[idx].data_type(); + new_columns.push(cast(&columns[start_idx], desired_datatype)?); + start_idx += 1; + } + } + let columns = new_columns; + // Ok(Some(RecordBatch::try_new(Arc::new(output_schema.to_owned()), columns)?)) + + let empty_arr = ScalarValue::iter_to_array(vec![ScalarValue::Null])?; + let n_res = indices[0] .iter() - .zip(output_schema.fields().iter()) - .map(|(col, desired_field)| cast(col, desired_field.data_type())) - .collect::>>()?; + .map(|(_, range)| range.1 - range.0) + .sum::() + + indices[1] + .iter() + .map(|(_, range)| range.1 - range.0) + .sum::() + + group_by_columns.len(); + let mut res = vec![empty_arr; n_res]; + for idx in 0..group_by_columns.len() { + res[idx] = group_by_columns[idx].clone(); + } + + let results = vec![columns, row_columns]; + for outer in 0..results.len() { + let mut start_idx = 0; + let cur_res = &results[outer]; + let cur_indices = &indices[outer]; + for (_idx, range) in cur_indices.iter() { + for res_idx in range.0..range.1 { + res[res_idx] = cur_res[start_idx].clone(); + start_idx += 1; + } + } + } - RecordBatch::try_new(Arc::new(output_schema.to_owned()), columns).map(Some) + Ok(Some(RecordBatch::try_new( + Arc::new(output_schema.to_owned()), + res, + )?)) } -fn read_as_batch(rows: &[Vec], schema: &Schema, row_type: RowType) -> Vec { +pub fn read_as_batch( + rows: &[Vec], + schema: &Schema, + row_type: RowType, +) -> Vec { let row_num = rows.len(); let mut output = MutableRecordBatch::new(row_num, Arc::new(schema.clone())); let mut row = RowReader::new(schema, row_type); @@ -583,3 +856,11 @@ fn read_as_batch(rows: &[Vec], schema: &Schema, row_type: RowType) -> Vec Result> { + // Copy group values out of arrays into `ScalarValue`s + grouping_set_values + .iter() + .map(|col| ScalarValue::try_from_array(col, row)) + .collect::>>() +} diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 41278e1208b78..c5952f1ab1ab6 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2340,3 +2340,325 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re Ok(()) } + +#[tokio::test] +#[ignore] +async fn test_accumulator_row_accumulator() -> Result<()> { + // let config = SessionConfig::new().with_target_partitions(1); + let config = SessionConfig::new(); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + // let sql = "SELECT MIN(c9), MAX(c11) + // FROM aggregate_test_100"; + // let sql = "SELECT MIN(c9), MIN(c13), MAX(c9), MAX(c13), AVG(c9) + // FROM aggregate_test_100"; + // let sql = "SELECT MIN(c9), MAX(c9) + // FROM aggregate_test_100 GROUP BY c1"; + // let sql = "SELECT AVG(c9) + // FROM aggregate_test_100 GROUP BY c1, c2"; + // let sql = "SELECT MIN(c13), MAX(c13), AVG(c9) + // FROM aggregate_test_100"; + + // let sql = "SELECT MIN(c9), MIN(c13), MAX(c9), MAX(c13), AVG(c9) + // FROM aggregate_test_100 GROUP BY c1"; + // let sql = "SELECT MIN(c13), MIN(c9) + // FROM aggregate_test_100"; + // let sql = "SELECT MIN(c9), MIN(c13) + // FROM aggregate_test_100"; + + let sql = + "SELECT c1, c2, MIN(c13), MIN(c9), MAX(c13), MAX(c9), AVG(c9), MIN(c13) as min1, COUNT(C9) + FROM aggregate_test_100 GROUP BY c1, c2 + ORDER BY c1, c2"; + + // let sql = + // "SELECT c1, c2, MIN(c13) + // FROM aggregate_test_100 GROUP BY c1, c2 + // ORDER BY c1, c2"; + + // let sql = "SELECT MIN(c13), MIN(c9), MAX(c13), MAX(c9), AVG(c9), MIN(c13) as min1 + // FROM aggregate_test_100"; + // let sql = "SELECT c1, MIN(c9), MIN(c13) + // FROM aggregate_test_100 GROUP BY c1"; + // let sql = "SELECT MIN(c13), MAX(c13) + // FROM aggregate_test_100 GROUP BY c1"; + // let sql = "SELECT MIN(c9), MAX(c9) + // FROM aggregate_test_100 GROUP BY c1"; + // let sql = "SELECT MIN(c9), MAX(c9), AVG(c9) + // FROM aggregate_test_100 GROUP BY c1"; + + let msg = format!("Creating logical plan for '{}'", sql); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + let formatted = displayable(physical_plan.as_ref()).indent().to_string(); + // // Only 1 SortExec was added + // let expected = { + // vec![ + // "ProjectionExec: expr=[c3@3 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@0 as sum2]", + // " GlobalLimitExec: skip=0, fetch=5", + // " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", + // " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow }]", + // " SortExec: [c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST]", + // ] + // }; + // + let actual: Vec<&str> = formatted.trim().lines().collect(); + let actual_len = actual.len(); + println!("{:#?}", actual); + // assert_eq!( + // expected, actual_trim_last, + // "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + // expected, actual + // ); + + let actual = execute_to_batches(&ctx, sql).await; + let expected = vec![ + "+----+----+--------------------------------+----------------------------+--------------------------------+----------------------------+----------------------------+--------------------------------+------------------------------+", + "| c1 | c2 | MIN(aggregate_test_100.c13) | MIN(aggregate_test_100.c9) | MAX(aggregate_test_100.c13) | MAX(aggregate_test_100.c9) | AVG(aggregate_test_100.c9) | min1 | COUNT(aggregate_test_100.c9) |", + "+----+----+--------------------------------+----------------------------+--------------------------------+----------------------------+----------------------------+--------------------------------+------------------------------+", + "| a | 1 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 774637006 | waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs | 4015442341 | 2437927011 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 5 |", + "| a | 2 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 145294611 | ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 | 3717551163 | 2267588664 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 3 |", + "| a | 3 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 431948861 | oLZ21P2JEDooxV1pU31cIxQHEeeoLu | 3998790955 | 2225685115.1666665 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 6 |", + "| a | 4 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 466439833 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 2502326480 | 1655431654 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 4 |", + "| a | 5 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 141047417 | QJYm7YRA3YetcBHI5wkMZeLXVmfuNy | 2496054700 | 1216992989.6666667 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 3 |", + "| b | 1 | 6FPJlLAcaQ5uokyOWZ9HGdLZObFvOZ | 1842680163 | akiiY5N0I44CMwEnBL6RTBk7BRkxEj | 4076864659 | 3064946272.6666665 | 6FPJlLAcaQ5uokyOWZ9HGdLZObFvOZ | 3 |", + "| b | 2 | 802bgTGl6Bk5TlkPYYTxp5JkKyaYUA | 1098639440 | okOkcWflkNXIy4R8LzmySyY1EC3sYd | 3455216719 | 2446327455.25 | 802bgTGl6Bk5TlkPYYTxp5JkKyaYUA | 4 |", + "| b | 3 | 6x93sxYioWuq5c9Kkk8oTAAORM7cH0 | 243203849 | MXhhH1Var3OzzJCtI9VNyYvA0q8UyJ | 1538863055 | 891033452 | 6x93sxYioWuq5c9Kkk8oTAAORM7cH0 | 2 |", + "| b | 4 | 52mKlRE3aHCBZtjECq6sY9OqVf8Dze | 326151275 | mhjME0zBHbrK6NMkytMTQzOssOa1gF | 3570297463 | 1819528986.6 | 52mKlRE3aHCBZtjECq6sY9OqVf8Dze | 5 |", + "| b | 5 | ALuRhobVWbnQTTWZdSOk0iVe8oYFhW | 974297360 | qnPOOmslCJaT45buUisMRnM0rc77EK | 3457053821 | 2501141166.8 | ALuRhobVWbnQTTWZdSOk0iVe8oYFhW | 5 |", + "| c | 1 | 2T3wSlHdEmASmO0xcXHnndkKEt6bz8 | 473294098 | mzbkwXKrPeZnxg2Kn1LRF5hYSsmksS | 3766999078 | 1721573913 | 2T3wSlHdEmASmO0xcXHnndkKEt6bz8 | 4 |", + "| c | 2 | 6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW | 141680161 | pLk3i59bZwd5KBZrI1FiweYTd5hteG | 3398507249 | 1703503487.7142856 | 6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW | 7 |", + "| c | 3 | 6oIXZuIPIqEoPBvFmbt2Nxy3tryGUE | 1088543984 | Ow5PGpfTm4dXCfTDsXAOTatXRoAydR | 3862393166 | 2446807725.75 | 6oIXZuIPIqEoPBvFmbt2Nxy3tryGUE | 4 |", + "| c | 4 | 8LIh0b6jmDGm87BmIyjdxNIpX4ugjD | 2306130875 | t6fQUjJejPcjc04wHvHTPe55S65B4V | 3593959807 | 3071416030.75 | 8LIh0b6jmDGm87BmIyjdxNIpX4ugjD | 4 |", + "| c | 5 | RilTlL1tKkPOUFuzmLydHAVZwv1OGl | 1229567292 | TTQUwpMNSXZqVBKAFvXu7OlWvKXJKX | 4268716378 | 2749141835 | RilTlL1tKkPOUFuzmLydHAVZwv1OGl | 2 |", + "| d | 1 | 4HX6feIvmNXBN7XGqgO4YVBkhu8GDI | 225513085 | y7C453hRWd4E7ImjNDWlpexB8nUqjh | 4216440507 | 2580450640.857143 | 4HX6feIvmNXBN7XGqgO4YVBkhu8GDI | 7 |", + "| d | 2 | 90gAtmGEeIqUTbo1ZrxCvWtsseukXC | 63044568 | lqhzgLsXZ8JhtpeeUWWNbMz8PHI705 | 4061635107 | 1708542649 | 90gAtmGEeIqUTbo1ZrxCvWtsseukXC | 3 |", + "| d | 3 | 1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO | 1824517658 | Z2sWcQr0qyCJRMHDpRy3aQr7PkHtkK | 2592330556 | 2170129047.3333335 | 1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO | 3 |", + "| d | 4 | 0og6hSkhbX8AC1ktFS4kounvTzy8Vo | 811650497 | f9ALCzwDAKmdu7Rk2msJaB1wxe5IBX | 3126475872 | 1824106822 | 0og6hSkhbX8AC1ktFS4kounvTzy8Vo | 3 |", + "| d | 5 | C2GT5KVyOPZpgKVl110TyZO0NcJ434 | 1365198901 | F7NSTjWvQJyBburN7CXRUlbgp2dIrA | 3373581039 | 2369389970 | C2GT5KVyOPZpgKVl110TyZO0NcJ434 | 2 |", + "| e | 1 | LiEBxds3X0Uw0lxiYjDqrkAaAwoiIW | 2610290479 | gpo8K5qtYePve6jyPt6xgJx4YOVjms | 4229654142 | 3223641897.6666665 | LiEBxds3X0Uw0lxiYjDqrkAaAwoiIW | 3 |", + "| e | 2 | BPtQMxnuSPpxMExYV9YkDa6cAN7GP3 | 1289293657 | xipQ93429ksjNcXPX5326VSg1xJZcW | 4144173353 | 2790023773 | BPtQMxnuSPpxMExYV9YkDa6cAN7GP3 | 5 |", + "| e | 3 | BsM5ZAYifRh5Lw3Y8X1r53I0cTJnfE | 559847112 | pTeu0WMjBRTaNRT15rLCuEh3tBJVc5 | 3759340273 | 2750454515.75 | BsM5ZAYifRh5Lw3Y8X1r53I0cTJnfE | 4 |", + "| e | 4 | 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 28774375 | oHJMNvWuunsIMIWFnYG31RCfkOo2V7 | 3521368277 | 1809406350.142857 | 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 7 |", + "| e | 5 | 56MZa5O1hVtX4c5sbnCfxuX5kDChqI | 662099130 | gxfHWUF8XgY2KdFxigxvNEXe2V2XMl | 2861911482 | 1762005306 | 56MZa5O1hVtX4c5sbnCfxuX5kDChqI | 2 |", + "+----+----+--------------------------------+----------------------------+--------------------------------+----------------------------+----------------------------+--------------------------------+------------------------------+", + ]; + assert_batches_eq!(expected, &actual); + + Ok(()) +} + +#[cfg(test)] +mod test_aggregate { + use arrow::array::{ArrayRef, Int32Array}; + use arrow::compute::concat_batches; + use arrow::datatypes::DataType; + use arrow::record_batch::RecordBatch; + use arrow::util::pretty::print_batches; + use datafusion::physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, + }; + use datafusion::physical_plan::collect; + use datafusion::physical_plan::memory::MemoryExec; + use datafusion::prelude::{SessionConfig, SessionContext}; + use datafusion_common::Result; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::expressions::{Avg, Count, Max, Min, Sum}; + use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; + use itertools::iproduct; + use rand::rngs::StdRng; + use rand::{Rng, SeedableRng}; + use std::sync::Arc; + use std::time::{Duration, Instant}; + use test_utils::add_empty_batches; + use tokio::runtime::Builder; + // use super::*; + + /// Return randomly sized record batches with: + /// two sorted int32 columns 'a', 'b' ranged from 0..len / DISTINCT as columns + /// two random int32 columns 'x', 'y' as other columns + fn make_staggered_batches( + len: usize, + distinct: usize, + random_seed: u64, + n_batch: usize, + ) -> Vec { + // use a random number generator to pick a random sized output + let mut rng = StdRng::seed_from_u64(random_seed); + // let mut input12: Vec<(i32, i32)> = vec![(0, 0); len]; + // let mut input3: Vec = vec![0; len]; + let mut input4: Vec = vec![0; len]; + // input12.iter_mut().for_each(|v| { + // *v = ( + // rng.gen_range(0..distinct) as i32, + // rng.gen_range(0..distinct) as i32, + // ) + // }); + // input3 + // .iter_mut() + // .for_each(|v| *v = rng.gen_range(0..distinct) as i32); + input4 + .iter_mut() + .for_each(|v| *v = rng.gen_range(0..distinct) as i32); + // // rng.fill(&mut input3[..]); + // // rng.fill(&mut input4[..]); + // input12.sort(); + // let input1 = + // Int32Array::from_iter_values(input12.clone().into_iter().map(|k| k.0)); + // let input2 = + // Int32Array::from_iter_values(input12.clone().into_iter().map(|k| k.1)); + // let input3 = Int32Array::from_iter_values(input3.into_iter()); + let input4 = Int32Array::from_iter_values(input4.into_iter()); + + // split into several record batches + let mut remainder = RecordBatch::try_from_iter(vec![ + ("a", Arc::new(input4.clone()) as ArrayRef), + ("b", Arc::new(input4.clone()) as ArrayRef), + ("x", Arc::new(input4.clone()) as ArrayRef), + ("y", Arc::new(input4) as ArrayRef), + ]) + .unwrap(); + + let mut batches = vec![]; + if STREAM { + while remainder.num_rows() > 0 { + let mut batch_size = rng.gen_range(0..n_batch); + if remainder.num_rows() < batch_size { + batch_size = remainder.num_rows() + } + batches.push(remainder.slice(0, batch_size)); + remainder = + remainder.slice(batch_size, remainder.num_rows() - batch_size); + } + } else { + while remainder.num_rows() > 0 { + let batch_size = rng.gen_range(0..remainder.num_rows() + 1); + batches.push(remainder.slice(0, batch_size)); + remainder = + remainder.slice(batch_size, remainder.num_rows() - batch_size); + } + } + add_empty_batches(batches, &mut rng) + } + + #[tokio::test] + #[ignore] + async fn test1() { + let n_trials = vec![10]; + let n_rows = vec![100, 1000, 100_000]; + let distincts = vec![10, 1000, 100_000, 100_000_000]; + let n_batches = vec![10, 100_000_000]; + // let n_trials = vec![10]; + // let n_rows = vec![100, 1000, 100_000]; + // let distincts = vec![10, 1000, 100_000, 100_000_000]; + // let n_batches = vec![10]; + for (n_trial, n_row, distinct, n_batch) in + iproduct!(n_trials, n_rows, distincts, n_batches) + { + let mut elapsed = vec![]; + for i in 0..n_trial { + let res = run_test(i, n_row, distinct, n_batch).await.unwrap(); + elapsed.push(res); + } + elapsed.sort(); + let tot_dur: Duration = elapsed.iter().sum(); + println!("------------------------------------"); + println!( + "n_row: {:?}, distinct: {:?}, n_batch: {:?}", + n_row, distinct, n_batch + ); + println!("elapsed mean: {:?}", tot_dur / elapsed.len() as u32); + println!("elapsed median: {:?}", elapsed[(n_trial / 2) as usize]); + println!("------------------------------------"); + } + } + + async fn run_test( + random_seed: u64, + n_row: usize, + distinct: usize, + n_batch: usize, + ) -> Result { + let in_data = + make_staggered_batches::(n_row, distinct, random_seed, n_batch); + let schema = in_data[0].schema(); + // let in_data = vec![concat_batches(&schema, &in_data).unwrap()]; + let in_exec = + Arc::new(MemoryExec::try_new(&[in_data], schema.clone(), None).unwrap()); + // print_batches(&in_data)?; + // let mut partitionby_exprs = vec![]; + // partitionby_exprs.push(col("a", &schema).unwrap()); + // let mut group_by_expr: Vec<(Arc, String)> = vec![]; + // group_by_expr.push((col("a", &schema).unwrap(), "dummy".to_string())); + // let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); + + let grouping_set = PhysicalGroupBy::new( + vec![(col("a", &schema)?, "a".to_string())], + vec![], + vec![vec![false]], + ); + + let aggregates: Vec> = vec![ + Arc::new(Sum::new( + col("x", &schema)?, + "Sum(x)".to_string(), + DataType::Int64, + )), + Arc::new(Min::new( + col("x", &schema)?, + "Min(x)".to_string(), + DataType::Int32, + )), + Arc::new(Max::new( + col("x", &schema)?, + "Max(x)".to_string(), + DataType::Int32, + )), + Arc::new(Avg::new( + col("x", &schema)?, + "Avg(x)".to_string(), + DataType::Float64, + )), + Arc::new(Sum::new( + col("y", &schema)?, + "Sum(y)".to_string(), + DataType::Int64, + )), + Arc::new(Min::new( + col("y", &schema)?, + "Min(y)".to_string(), + DataType::Int32, + )), + Arc::new(Max::new( + col("y", &schema)?, + "Max(y)".to_string(), + DataType::Int32, + )), + Arc::new(Avg::new( + col("y", &schema)?, + "Avg(y)".to_string(), + DataType::Float64, + )), + ]; + // println!("aggregates:{:?}", aggregates); + + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + grouping_set, + aggregates, + in_exec, + schema, + )?) as _; + + let session_config = SessionConfig::new().with_batch_size(50); + let ctx = SessionContext::with_config(session_config); + let task_ctx = ctx.task_ctx(); + let now = Instant::now(); + let collected_running = collect(aggregate_exec, task_ctx.clone()).await.unwrap(); + let elapsed = now.elapsed(); + // println!("Elapsed: {:.2?}", now.elapsed()); + // println!("Elapsed: {:.2?}", now_start.elapsed()); + // print_batches(&collected_running)?; + Ok(elapsed) + } +} diff --git a/datafusion/physical-expr/src/aggregate/count.rs b/datafusion/physical-expr/src/aggregate/count.rs index 813952117af11..4031d6c7c5c80 100644 --- a/datafusion/physical-expr/src/aggregate/count.rs +++ b/datafusion/physical-expr/src/aggregate/count.rs @@ -198,7 +198,10 @@ impl RowAccumulator for CountRowAccumulator { } fn evaluate(&self, accessor: &RowAccessor) -> Result { - Ok(accessor.get_as_scalar(&DataType::Int64, self.state_index)) + Ok(match accessor.get_u64_opt(self.state_index()) { + Some(cnt) => ScalarValue::Int64(Some(cnt as i64)), + None => ScalarValue::Int64(Some(0)), + }) } #[inline(always)] diff --git a/test-utils/src/lib.rs b/test-utils/src/lib.rs index 4002a49cf585b..dfd878275181c 100644 --- a/test-utils/src/lib.rs +++ b/test-utils/src/lib.rs @@ -50,7 +50,10 @@ pub fn partitions_to_sorted_vec(partitions: &[Vec]) -> Vec, rng: &mut StdRng) -> Vec { +pub fn add_empty_batches( + batches: Vec, + rng: &mut StdRng, +) -> Vec { let schema = batches[0].schema(); batches From 077c5835250182d9d89e3bd8f1ce1b5f1fa91c22 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 9 Jan 2023 17:31:40 +0300 Subject: [PATCH 02/19] accumulator time fix --- .../src/physical_plan/aggregates/row_hash.rs | 20 +++++-------------- datafusion/core/tests/sql/window.rs | 6 +++--- 2 files changed, 8 insertions(+), 18 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 78c1f731b0ed1..a99c603ae81e3 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -750,22 +750,14 @@ fn create_batch_from_map( } } - // make group states mutable - let mut accumulator_set_vec: Vec<_> = row_aggr_state - .group_states - .iter() - .map(|group_state| &group_state.accumulator_set) - .collect(); - // next, output aggregates: either intermediate state or final output - let mut start_idx = 0; for (x, &state_len) in acc_data_types.iter().enumerate() { for y in 0..state_len { match mode { AggregateMode::Partial => { let res = ScalarValue::iter_to_array( - accumulator_set_vec.iter().map(|accumulator_set| { - accumulator_set[x] + row_aggr_state.group_states.iter().map(|row_group_state| { + row_group_state.accumulator_set[x] .state() .map(|x| x[y].clone()) .expect("unexpected accumulator state in hash aggregate") @@ -776,12 +768,10 @@ fn create_batch_from_map( } AggregateMode::Final | AggregateMode::FinalPartitioned => { let res = ScalarValue::iter_to_array( - accumulator_set_vec.iter_mut().map(|accumulator_set| { - let res = accumulator_set[x] + row_aggr_state.group_states.iter().map(|row_group_state| { + row_group_state.accumulator_set[x] .evaluate() - .expect("unexpected accumulator state in hash aggregate"); - start_idx += 1; - res + .expect("unexpected accumulator state in hash aggregate") }), )?; columns.push(res); diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index c5952f1ab1ab6..1bcf7c67e97c1 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2549,9 +2549,9 @@ mod test_aggregate { let distincts = vec![10, 1000, 100_000, 100_000_000]; let n_batches = vec![10, 100_000_000]; // let n_trials = vec![10]; - // let n_rows = vec![100, 1000, 100_000]; - // let distincts = vec![10, 1000, 100_000, 100_000_000]; - // let n_batches = vec![10]; + // let n_rows = vec![100_000]; + // let distincts = vec![100_000_000]; + // let n_batches = vec![100_000_000]; for (n_trial, n_row, distinct, n_batch) in iproduct!(n_trials, n_rows, distincts, n_batches) { From c61f1e57f1247a76ed726b50341b3a17bcdf699e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 10 Jan 2023 10:06:18 +0300 Subject: [PATCH 03/19] resolve linter errors --- .../core/src/physical_plan/aggregates/mod.rs | 29 +++------- .../physical_plan/aggregates/no_grouping.rs | 54 ++++--------------- .../src/physical_plan/aggregates/row_hash.rs | 51 +++++------------- datafusion/core/tests/sql/window.rs | 48 +++-------------- 4 files changed, 33 insertions(+), 149 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 2d0b7d919beaa..fb06c60e2ba0f 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -38,7 +38,6 @@ use datafusion_physical_expr::{ use std::any::Any; use std::collections::HashMap; -use parquet::schema::printer::print_schema; use std::sync::Arc; mod no_grouping; @@ -54,7 +53,7 @@ use datafusion_physical_expr::equivalence::project_equivalence_properties; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::normalize_out_expr_with_alias_schema; use datafusion_row::accessor::RowAccessor; -use datafusion_row::{row_supported, RowType}; +use datafusion_row::RowType; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -272,12 +271,6 @@ impl AggregateExec { self.input_schema.clone() } - fn row_aggregate_supported(&self) -> bool { - let group_schema = group_schema(&self.schema, self.group_by.expr.len()); - row_supported(&group_schema, RowType::Compact) - && accumulator_v2_supported(&self.aggr_expr) - } - fn execute_typed( &self, partition: usize, @@ -285,7 +278,6 @@ impl AggregateExec { ) -> Result { let batch_size = context.session_config().batch_size(); let input = self.input.execute(partition, Arc::clone(&context))?; - const VERSION: usize = 1; let baseline_metrics = BaselineMetrics::new(&self.metrics, partition); if self.group_by.expr.is_empty() { Ok(StreamType::AggregateStream(AggregateStream::new( @@ -596,12 +588,6 @@ fn create_accumulators( .collect::>>() } -fn accumulator_v2_supported(aggr_expr: &[Arc]) -> bool { - aggr_expr - .iter() - .all(|expr| expr.row_accumulator_supported()) -} - fn create_accumulators_v2( aggr_expr: &[Arc], ) -> datafusion_common::Result> { @@ -625,9 +611,9 @@ fn finalize_aggregation( aggr_schema: &Schema, aggr_state: &mut RowAggregationState, output_schema: &Schema, - indices: &Vec>, + indices: &[Vec<(usize, (usize, usize))>], ) -> datafusion_common::Result> { - let mut acc_res = match mode { + let acc_res = match mode { AggregateMode::Partial => { // build the vector of states let a = accumulators @@ -693,15 +679,12 @@ fn finalize_aggregation( .sum::(); let mut res = vec![empty_arr; n_res]; let results = vec![acc_res, columns]; - for outer in [0, 1] { + for (outer, cur_res) in results.into_iter().enumerate() { let mut start_idx = 0; - let cur_res = &results[outer]; let cur_indices = &indices[outer]; - // println!("cur res:{:?}", cur_res); - // println!("cur_indices:{:?}", cur_indices); for (_idx, range) in cur_indices.iter() { - for res_idx in range.0..range.1 { - res[res_idx] = cur_res[start_idx].clone(); + for elem in res.iter_mut().take(range.1).skip(range.0) { + *elem = cur_res[start_idx].clone(); start_idx += 1; } } diff --git a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs index f00d79652b1a9..6f8979434b558 100644 --- a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs +++ b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs @@ -24,8 +24,6 @@ use crate::physical_plan::aggregates::{ }; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; -use arrow::array::{Array, ArrayRef}; -use arrow::compute; use arrow::datatypes::SchemaRef; use arrow::error::{ArrowError, Result as ArrowResult}; use arrow::record_batch::RecordBatch; @@ -87,14 +85,12 @@ impl AggregateStream { let mut row_agg_indices = vec![]; let mut normal_agg_indices = vec![]; let mut start_idx = 0; - for idx in 0..aggr_expr.len() { + for (idx, expr) in aggr_expr.iter().enumerate() { let n_field = match mode { - AggregateMode::Partial => aggr_expr[idx].state_fields()?.len(), + AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; - // println!("aggr_expr[idx]: {:?}", aggr_expr[idx]); - // println!("n_field:{:?}", n_field); - if aggr_expr[idx].row_accumulator_supported() { + if expr.row_accumulator_supported() { row_agg_indices.push((idx, (start_idx, start_idx + n_field))); } else { normal_agg_indices.push((idx, (start_idx, start_idx + n_field))); @@ -102,7 +98,6 @@ impl AggregateStream { start_idx += n_field; } let indices = vec![normal_agg_indices, row_agg_indices]; - println!("indices:{:?}", indices); let row_aggr_exprs = aggr_expr .clone() .into_iter() @@ -127,7 +122,7 @@ impl AggregateStream { group_by_values: Box::new([]), aggregation_buffer: vec![0; row_aggr_layout.fixed_part_width()], accumulator_set: vec![], - indices: vec![0 as u32], // 1.3 + indices: vec![0_u32], // 1.3 }; let row_aggr_state = RowAggregationState { reservation, @@ -144,14 +139,6 @@ impl AggregateStream { for (idx, _) in &indices[1] { row_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) } - // let normal_aggregate_expressions = aggregate_expressions(&normal_aggr_exprs, &mode, 0)?; - // let row_aggregate_expressions = aggregate_expressions(&row_aggr_exprs, &mode, 0)?; - // println!("all_aggregate_expressions:{:?}", all_aggregate_expressions); - // println!( - // "normal_aggregate_expressions:{:?}", - // normal_aggregate_expressions - // ); - // println!("row_aggregate_expressions:{:?}", row_aggregate_expressions); let normal_accumulators = create_accumulators(&normal_aggr_exprs)?; let row_accumulators = create_accumulators_v2(&row_aggr_exprs)?; @@ -269,6 +256,7 @@ impl RecordBatchStream for AggregateStream { /// If successfull, this returns the additional number of bytes that were allocated during this process. /// /// TODO: Make this a member function +#[allow(clippy::too_many_arguments)] fn aggregate_batch( mode: &AggregateMode, batch: &RecordBatch, @@ -279,30 +267,9 @@ fn aggregate_batch( row_aggr_state: &mut RowAggregationState, state_layout: Arc, ) -> Result { - let RowAggregationState { - map, group_states, .. - } = row_aggr_state; + let RowAggregationState { group_states, .. } = row_aggr_state; let group_state = &mut group_states[0]; - let aggr_input_values = evaluate_many(row_expressions, &batch)?; - // println!("aggr_input_values:{:?}", aggr_input_values); - // // `Take` all values based on indices into Arrays - // let values: Vec>> = aggr_input_values - // .iter() - // .map(|array| { - // array - // .iter() - // .map(|array| { - // compute::take( - // array.as_ref(), - // &batch_indices, - // None, // None: no index check - // ) - // .unwrap() - // }) - // .collect() - // // 2.3 - // }) - // .collect(); + let aggr_input_values = evaluate_many(row_expressions, batch)?; row_accumulators .iter_mut() .zip(aggr_input_values.iter()) @@ -310,14 +277,13 @@ fn aggregate_batch( .try_for_each(|(accumulator, values)| { let mut state_accessor = RowAccessor::new_from_layout(state_layout.clone()); state_accessor.point_to(0, group_state.aggregation_buffer.as_mut_slice()); - // println!("row acc values:{:?}, mode:{:?}", values, mode); match mode { AggregateMode::Partial => { - accumulator.update_batch(&values, &mut state_accessor) + accumulator.update_batch(values, &mut state_accessor) } AggregateMode::FinalPartitioned | AggregateMode::Final => { // note: the aggregation here is over states, not values, thus the merge - accumulator.merge_batch(&values, &mut state_accessor) + accumulator.merge_batch(values, &mut state_accessor) } } }) @@ -334,8 +300,6 @@ fn aggregate_batch( // 1.3 update / merge accumulators with the expressions' values // 1.1 - // println!("expressions:{:?}", expressions); - // println!("row_expressions:{:?}", row_expressions); accumulators .iter_mut() .zip(expressions) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index a99c603ae81e3..2b1411b865e5a 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -17,7 +17,6 @@ //! Hash aggregation through row format -use std::collections::VecDeque; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; @@ -33,13 +32,11 @@ use crate::physical_plan::aggregates::{ evaluate_group_by, evaluate_many, group_schema, AccumulatorItem, AccumulatorItemV2, AggregateMode, PhysicalGroupBy, }; -use crate::physical_plan::hash_utils::create_row_hashes; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use crate::physical_plan::common::transpose; use arrow::compute::cast; use arrow::datatypes::Schema; use arrow::{array::ArrayRef, compute}; @@ -54,7 +51,6 @@ use datafusion_physical_expr::hash_utils::create_hashes; use datafusion_row::accessor::RowAccessor; use datafusion_row::layout::RowLayout; use datafusion_row::reader::{read_row, RowReader}; -use datafusion_row::writer::{write_row, RowWriter}; use datafusion_row::{row_supported, MutableRecordBatch, RowType}; use hashbrown::raw::RawTable; @@ -94,10 +90,8 @@ struct GroupedHashAggregateStreamV2Inner { row_aggregate_expressions: Vec>>, group_by: PhysicalGroupBy, - normal_accumulators: Vec, row_accumulators: Vec, - group_schema: SchemaRef, row_aggr_schema: SchemaRef, row_aggr_layout: Arc, @@ -143,12 +137,12 @@ impl GroupedHashAggregateStreamV2 { let mut row_agg_indices = vec![]; let mut normal_agg_indices = vec![]; let mut start_idx = group_by.expr.len(); - for idx in 0..aggr_expr.len() { + for (idx, expr) in aggr_expr.iter().enumerate() { let n_field = match mode { - AggregateMode::Partial => aggr_expr[idx].state_fields()?.len(), + AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; - if is_supported(&aggr_expr[idx], &group_schema) { + if is_supported(expr, &group_schema) { row_agg_indices.push((idx, (start_idx, start_idx + n_field))); } else { normal_agg_indices.push((idx, (start_idx, start_idx + n_field))); @@ -182,7 +176,6 @@ impl GroupedHashAggregateStreamV2 { } // let accumulators = aggregates::create_accumulators_v2(&aggr_expr)?; - let normal_accumulators = aggregates::create_accumulators(&normal_aggr_exprs)?; let row_accumulators = aggregates::create_accumulators_v2(&row_aggr_exprs)?; let row_aggr_schema = aggr_state_schema(&row_aggr_exprs)?; @@ -207,9 +200,7 @@ impl GroupedHashAggregateStreamV2 { input, group_by, normal_aggr_expr: normal_aggr_exprs, - normal_accumulators, row_accumulators, - group_schema, row_aggr_schema, row_aggr_layout, baseline_metrics, @@ -236,7 +227,6 @@ impl GroupedHashAggregateStreamV2 { &this.group_by, &this.normal_aggr_expr, &mut this.row_accumulators, - &this.group_schema, this.row_aggr_layout.clone(), batch, &mut this.row_aggr_state, @@ -261,7 +251,6 @@ impl GroupedHashAggregateStreamV2 { let timer = this.baseline_metrics.elapsed_compute().timer(); let result = create_batch_from_map( &this.mode, - &this.group_schema, &this.row_aggr_schema, this.batch_size, this.row_group_skip_position, @@ -329,7 +318,6 @@ fn group_aggregate_batch( grouping_set: &PhysicalGroupBy, normal_aggr_expr: &[Arc], row_accumulators: &mut [AccumulatorItemV2], - group_schema: &Schema, state_layout: Arc, batch: RecordBatch, aggr_state: &mut RowAggregationState, @@ -363,7 +351,7 @@ fn group_aggregate_batch( // 1.1 Calculate the group keys for the group values let mut batch_hashes = vec![0; batch.num_rows()]; - create_hashes(&group_values, random_state, &mut batch_hashes)?; + create_hashes(group_values, random_state, &mut batch_hashes)?; for (row, hash) in batch_hashes.into_iter().enumerate() { let entry = row_map.get_mut(hash, |(_hash, group_idx)| { @@ -530,7 +518,7 @@ fn group_aggregate_batch( } }) // 2.5 - .and({ Ok(()) })?; + .and(Ok(()))?; // normal accumulators group_state .accumulator_set @@ -619,30 +607,17 @@ impl std::fmt::Debug for RowAggregationState { } } -/// Create grouping rows -fn create_group_rows(arrays: Vec, schema: &Schema) -> Vec> { - let mut writer = RowWriter::new(schema, RowType::Compact); - let mut results = vec![]; - for cur_row in 0..arrays[0].len() { - write_row(&mut writer, cur_row, schema, &arrays); - results.push(writer.get_row().to_vec()); - writer.reset() - } - results -} - /// Create a RecordBatch with all group keys and accumulator' states or values. #[allow(clippy::too_many_arguments)] fn create_batch_from_map( mode: &AggregateMode, - group_schema: &Schema, aggr_schema: &Schema, batch_size: usize, skip_items: usize, row_aggr_state: &mut RowAggregationState, row_accumulators: &mut [AccumulatorItemV2], output_schema: &Schema, - indices: &Vec>, + indices: &[Vec<(usize, (usize, usize))>], num_group_expr: usize, ) -> ArrowResult> { if skip_items > row_aggr_state.group_states.len() { @@ -670,7 +645,7 @@ fn create_batch_from_map( )))); } // First, output all group by exprs - let mut group_by_columns = (0..num_group_expr) + let group_by_columns = (0..num_group_expr) .map(|idx| { ScalarValue::iter_to_array(group_buffers.iter().map(|x| x[idx].clone())) }) @@ -794,7 +769,6 @@ fn create_batch_from_map( } } let columns = new_columns; - // Ok(Some(RecordBatch::try_new(Arc::new(output_schema.to_owned()), columns)?)) let empty_arr = ScalarValue::iter_to_array(vec![ScalarValue::Null])?; let n_res = indices[0] @@ -807,18 +781,17 @@ fn create_batch_from_map( .sum::() + group_by_columns.len(); let mut res = vec![empty_arr; n_res]; - for idx in 0..group_by_columns.len() { - res[idx] = group_by_columns[idx].clone(); + for (idx, column) in group_by_columns.into_iter().enumerate() { + res[idx] = column; } let results = vec![columns, row_columns]; - for outer in 0..results.len() { + for (outer, cur_res) in results.into_iter().enumerate() { let mut start_idx = 0; - let cur_res = &results[outer]; let cur_indices = &indices[outer]; for (_idx, range) in cur_indices.iter() { - for res_idx in range.0..range.1 { - res[res_idx] = cur_res[start_idx].clone(); + for elem in res.iter_mut().take(range.1).skip(range.0) { + *elem = cur_res[start_idx].clone(); start_idx += 1; } } diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 1bcf7c67e97c1..98b8a06e1f507 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2403,13 +2403,7 @@ async fn test_accumulator_row_accumulator() -> Result<()> { // }; // let actual: Vec<&str> = formatted.trim().lines().collect(); - let actual_len = actual.len(); println!("{:#?}", actual); - // assert_eq!( - // expected, actual_trim_last, - // "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - // expected, actual - // ); let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ @@ -2450,29 +2444,19 @@ async fn test_accumulator_row_accumulator() -> Result<()> { #[cfg(test)] mod test_aggregate { - use arrow::array::{ArrayRef, Int32Array}; - use arrow::compute::concat_batches; - use arrow::datatypes::DataType; - use arrow::record_batch::RecordBatch; - use arrow::util::pretty::print_batches; - use datafusion::physical_plan::aggregates::{ - AggregateExec, AggregateMode, PhysicalGroupBy, - }; - use datafusion::physical_plan::collect; + use super::*; + use datafusion::physical_plan::aggregates::AggregateExec; + use datafusion::physical_plan::aggregates::{AggregateMode, PhysicalGroupBy}; use datafusion::physical_plan::memory::MemoryExec; - use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::Result; use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::expressions::{Avg, Count, Max, Min, Sum}; - use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; + use datafusion_physical_expr::expressions::{Avg, Max, Min, Sum}; + use datafusion_physical_expr::AggregateExpr; use itertools::iproduct; use rand::rngs::StdRng; use rand::{Rng, SeedableRng}; - use std::sync::Arc; use std::time::{Duration, Instant}; use test_utils::add_empty_batches; - use tokio::runtime::Builder; - // use super::*; /// Return randomly sized record batches with: /// two sorted int32 columns 'a', 'b' ranged from 0..len / DISTINCT as columns @@ -2485,29 +2469,10 @@ mod test_aggregate { ) -> Vec { // use a random number generator to pick a random sized output let mut rng = StdRng::seed_from_u64(random_seed); - // let mut input12: Vec<(i32, i32)> = vec![(0, 0); len]; - // let mut input3: Vec = vec![0; len]; let mut input4: Vec = vec![0; len]; - // input12.iter_mut().for_each(|v| { - // *v = ( - // rng.gen_range(0..distinct) as i32, - // rng.gen_range(0..distinct) as i32, - // ) - // }); - // input3 - // .iter_mut() - // .for_each(|v| *v = rng.gen_range(0..distinct) as i32); input4 .iter_mut() .for_each(|v| *v = rng.gen_range(0..distinct) as i32); - // // rng.fill(&mut input3[..]); - // // rng.fill(&mut input4[..]); - // input12.sort(); - // let input1 = - // Int32Array::from_iter_values(input12.clone().into_iter().map(|k| k.0)); - // let input2 = - // Int32Array::from_iter_values(input12.clone().into_iter().map(|k| k.1)); - // let input3 = Int32Array::from_iter_values(input3.into_iter()); let input4 = Int32Array::from_iter_values(input4.into_iter()); // split into several record batches @@ -2640,7 +2605,6 @@ mod test_aggregate { DataType::Float64, )), ]; - // println!("aggregates:{:?}", aggregates); let aggregate_exec = Arc::new(AggregateExec::try_new( AggregateMode::Partial, @@ -2654,7 +2618,7 @@ mod test_aggregate { let ctx = SessionContext::with_config(session_config); let task_ctx = ctx.task_ctx(); let now = Instant::now(); - let collected_running = collect(aggregate_exec, task_ctx.clone()).await.unwrap(); + let _collected_running = collect(aggregate_exec, task_ctx.clone()).await.unwrap(); let elapsed = now.elapsed(); // println!("Elapsed: {:.2?}", now.elapsed()); // println!("Elapsed: {:.2?}", now_start.elapsed()); From 6039b891bf6005141264bfbcc73f8529a680a91f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 10 Jan 2023 10:43:18 +0300 Subject: [PATCH 04/19] remove unnecessary changes --- datafusion/core/tests/sql/window.rs | 286 ---------------------------- 1 file changed, 286 deletions(-) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 98b8a06e1f507..41278e1208b78 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2340,289 +2340,3 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re Ok(()) } - -#[tokio::test] -#[ignore] -async fn test_accumulator_row_accumulator() -> Result<()> { - // let config = SessionConfig::new().with_target_partitions(1); - let config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); - register_aggregate_csv(&ctx).await?; - // let sql = "SELECT MIN(c9), MAX(c11) - // FROM aggregate_test_100"; - // let sql = "SELECT MIN(c9), MIN(c13), MAX(c9), MAX(c13), AVG(c9) - // FROM aggregate_test_100"; - // let sql = "SELECT MIN(c9), MAX(c9) - // FROM aggregate_test_100 GROUP BY c1"; - // let sql = "SELECT AVG(c9) - // FROM aggregate_test_100 GROUP BY c1, c2"; - // let sql = "SELECT MIN(c13), MAX(c13), AVG(c9) - // FROM aggregate_test_100"; - - // let sql = "SELECT MIN(c9), MIN(c13), MAX(c9), MAX(c13), AVG(c9) - // FROM aggregate_test_100 GROUP BY c1"; - // let sql = "SELECT MIN(c13), MIN(c9) - // FROM aggregate_test_100"; - // let sql = "SELECT MIN(c9), MIN(c13) - // FROM aggregate_test_100"; - - let sql = - "SELECT c1, c2, MIN(c13), MIN(c9), MAX(c13), MAX(c9), AVG(c9), MIN(c13) as min1, COUNT(C9) - FROM aggregate_test_100 GROUP BY c1, c2 - ORDER BY c1, c2"; - - // let sql = - // "SELECT c1, c2, MIN(c13) - // FROM aggregate_test_100 GROUP BY c1, c2 - // ORDER BY c1, c2"; - - // let sql = "SELECT MIN(c13), MIN(c9), MAX(c13), MAX(c9), AVG(c9), MIN(c13) as min1 - // FROM aggregate_test_100"; - // let sql = "SELECT c1, MIN(c9), MIN(c13) - // FROM aggregate_test_100 GROUP BY c1"; - // let sql = "SELECT MIN(c13), MAX(c13) - // FROM aggregate_test_100 GROUP BY c1"; - // let sql = "SELECT MIN(c9), MAX(c9) - // FROM aggregate_test_100 GROUP BY c1"; - // let sql = "SELECT MIN(c9), MAX(c9), AVG(c9) - // FROM aggregate_test_100 GROUP BY c1"; - - let msg = format!("Creating logical plan for '{}'", sql); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - let formatted = displayable(physical_plan.as_ref()).indent().to_string(); - // // Only 1 SortExec was added - // let expected = { - // vec![ - // "ProjectionExec: expr=[c3@3 as c3, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as sum1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c3] ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@0 as sum2]", - // " GlobalLimitExec: skip=0, fetch=5", - // " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - // " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow }]", - // " SortExec: [c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST]", - // ] - // }; - // - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - - let actual = execute_to_batches(&ctx, sql).await; - let expected = vec![ - "+----+----+--------------------------------+----------------------------+--------------------------------+----------------------------+----------------------------+--------------------------------+------------------------------+", - "| c1 | c2 | MIN(aggregate_test_100.c13) | MIN(aggregate_test_100.c9) | MAX(aggregate_test_100.c13) | MAX(aggregate_test_100.c9) | AVG(aggregate_test_100.c9) | min1 | COUNT(aggregate_test_100.c9) |", - "+----+----+--------------------------------+----------------------------+--------------------------------+----------------------------+----------------------------+--------------------------------+------------------------------+", - "| a | 1 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 774637006 | waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs | 4015442341 | 2437927011 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 5 |", - "| a | 2 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 145294611 | ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 | 3717551163 | 2267588664 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 3 |", - "| a | 3 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 431948861 | oLZ21P2JEDooxV1pU31cIxQHEeeoLu | 3998790955 | 2225685115.1666665 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 6 |", - "| a | 4 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 466439833 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 2502326480 | 1655431654 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 4 |", - "| a | 5 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 141047417 | QJYm7YRA3YetcBHI5wkMZeLXVmfuNy | 2496054700 | 1216992989.6666667 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 3 |", - "| b | 1 | 6FPJlLAcaQ5uokyOWZ9HGdLZObFvOZ | 1842680163 | akiiY5N0I44CMwEnBL6RTBk7BRkxEj | 4076864659 | 3064946272.6666665 | 6FPJlLAcaQ5uokyOWZ9HGdLZObFvOZ | 3 |", - "| b | 2 | 802bgTGl6Bk5TlkPYYTxp5JkKyaYUA | 1098639440 | okOkcWflkNXIy4R8LzmySyY1EC3sYd | 3455216719 | 2446327455.25 | 802bgTGl6Bk5TlkPYYTxp5JkKyaYUA | 4 |", - "| b | 3 | 6x93sxYioWuq5c9Kkk8oTAAORM7cH0 | 243203849 | MXhhH1Var3OzzJCtI9VNyYvA0q8UyJ | 1538863055 | 891033452 | 6x93sxYioWuq5c9Kkk8oTAAORM7cH0 | 2 |", - "| b | 4 | 52mKlRE3aHCBZtjECq6sY9OqVf8Dze | 326151275 | mhjME0zBHbrK6NMkytMTQzOssOa1gF | 3570297463 | 1819528986.6 | 52mKlRE3aHCBZtjECq6sY9OqVf8Dze | 5 |", - "| b | 5 | ALuRhobVWbnQTTWZdSOk0iVe8oYFhW | 974297360 | qnPOOmslCJaT45buUisMRnM0rc77EK | 3457053821 | 2501141166.8 | ALuRhobVWbnQTTWZdSOk0iVe8oYFhW | 5 |", - "| c | 1 | 2T3wSlHdEmASmO0xcXHnndkKEt6bz8 | 473294098 | mzbkwXKrPeZnxg2Kn1LRF5hYSsmksS | 3766999078 | 1721573913 | 2T3wSlHdEmASmO0xcXHnndkKEt6bz8 | 4 |", - "| c | 2 | 6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW | 141680161 | pLk3i59bZwd5KBZrI1FiweYTd5hteG | 3398507249 | 1703503487.7142856 | 6WfVFBVGJSQb7FhA7E0lBwdvjfZnSW | 7 |", - "| c | 3 | 6oIXZuIPIqEoPBvFmbt2Nxy3tryGUE | 1088543984 | Ow5PGpfTm4dXCfTDsXAOTatXRoAydR | 3862393166 | 2446807725.75 | 6oIXZuIPIqEoPBvFmbt2Nxy3tryGUE | 4 |", - "| c | 4 | 8LIh0b6jmDGm87BmIyjdxNIpX4ugjD | 2306130875 | t6fQUjJejPcjc04wHvHTPe55S65B4V | 3593959807 | 3071416030.75 | 8LIh0b6jmDGm87BmIyjdxNIpX4ugjD | 4 |", - "| c | 5 | RilTlL1tKkPOUFuzmLydHAVZwv1OGl | 1229567292 | TTQUwpMNSXZqVBKAFvXu7OlWvKXJKX | 4268716378 | 2749141835 | RilTlL1tKkPOUFuzmLydHAVZwv1OGl | 2 |", - "| d | 1 | 4HX6feIvmNXBN7XGqgO4YVBkhu8GDI | 225513085 | y7C453hRWd4E7ImjNDWlpexB8nUqjh | 4216440507 | 2580450640.857143 | 4HX6feIvmNXBN7XGqgO4YVBkhu8GDI | 7 |", - "| d | 2 | 90gAtmGEeIqUTbo1ZrxCvWtsseukXC | 63044568 | lqhzgLsXZ8JhtpeeUWWNbMz8PHI705 | 4061635107 | 1708542649 | 90gAtmGEeIqUTbo1ZrxCvWtsseukXC | 3 |", - "| d | 3 | 1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO | 1824517658 | Z2sWcQr0qyCJRMHDpRy3aQr7PkHtkK | 2592330556 | 2170129047.3333335 | 1aOcrEGd0cOqZe2I5XBOm0nDcwtBZO | 3 |", - "| d | 4 | 0og6hSkhbX8AC1ktFS4kounvTzy8Vo | 811650497 | f9ALCzwDAKmdu7Rk2msJaB1wxe5IBX | 3126475872 | 1824106822 | 0og6hSkhbX8AC1ktFS4kounvTzy8Vo | 3 |", - "| d | 5 | C2GT5KVyOPZpgKVl110TyZO0NcJ434 | 1365198901 | F7NSTjWvQJyBburN7CXRUlbgp2dIrA | 3373581039 | 2369389970 | C2GT5KVyOPZpgKVl110TyZO0NcJ434 | 2 |", - "| e | 1 | LiEBxds3X0Uw0lxiYjDqrkAaAwoiIW | 2610290479 | gpo8K5qtYePve6jyPt6xgJx4YOVjms | 4229654142 | 3223641897.6666665 | LiEBxds3X0Uw0lxiYjDqrkAaAwoiIW | 3 |", - "| e | 2 | BPtQMxnuSPpxMExYV9YkDa6cAN7GP3 | 1289293657 | xipQ93429ksjNcXPX5326VSg1xJZcW | 4144173353 | 2790023773 | BPtQMxnuSPpxMExYV9YkDa6cAN7GP3 | 5 |", - "| e | 3 | BsM5ZAYifRh5Lw3Y8X1r53I0cTJnfE | 559847112 | pTeu0WMjBRTaNRT15rLCuEh3tBJVc5 | 3759340273 | 2750454515.75 | BsM5ZAYifRh5Lw3Y8X1r53I0cTJnfE | 4 |", - "| e | 4 | 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 28774375 | oHJMNvWuunsIMIWFnYG31RCfkOo2V7 | 3521368277 | 1809406350.142857 | 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 7 |", - "| e | 5 | 56MZa5O1hVtX4c5sbnCfxuX5kDChqI | 662099130 | gxfHWUF8XgY2KdFxigxvNEXe2V2XMl | 2861911482 | 1762005306 | 56MZa5O1hVtX4c5sbnCfxuX5kDChqI | 2 |", - "+----+----+--------------------------------+----------------------------+--------------------------------+----------------------------+----------------------------+--------------------------------+------------------------------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[cfg(test)] -mod test_aggregate { - use super::*; - use datafusion::physical_plan::aggregates::AggregateExec; - use datafusion::physical_plan::aggregates::{AggregateMode, PhysicalGroupBy}; - use datafusion::physical_plan::memory::MemoryExec; - use datafusion_common::Result; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr::expressions::{Avg, Max, Min, Sum}; - use datafusion_physical_expr::AggregateExpr; - use itertools::iproduct; - use rand::rngs::StdRng; - use rand::{Rng, SeedableRng}; - use std::time::{Duration, Instant}; - use test_utils::add_empty_batches; - - /// Return randomly sized record batches with: - /// two sorted int32 columns 'a', 'b' ranged from 0..len / DISTINCT as columns - /// two random int32 columns 'x', 'y' as other columns - fn make_staggered_batches( - len: usize, - distinct: usize, - random_seed: u64, - n_batch: usize, - ) -> Vec { - // use a random number generator to pick a random sized output - let mut rng = StdRng::seed_from_u64(random_seed); - let mut input4: Vec = vec![0; len]; - input4 - .iter_mut() - .for_each(|v| *v = rng.gen_range(0..distinct) as i32); - let input4 = Int32Array::from_iter_values(input4.into_iter()); - - // split into several record batches - let mut remainder = RecordBatch::try_from_iter(vec![ - ("a", Arc::new(input4.clone()) as ArrayRef), - ("b", Arc::new(input4.clone()) as ArrayRef), - ("x", Arc::new(input4.clone()) as ArrayRef), - ("y", Arc::new(input4) as ArrayRef), - ]) - .unwrap(); - - let mut batches = vec![]; - if STREAM { - while remainder.num_rows() > 0 { - let mut batch_size = rng.gen_range(0..n_batch); - if remainder.num_rows() < batch_size { - batch_size = remainder.num_rows() - } - batches.push(remainder.slice(0, batch_size)); - remainder = - remainder.slice(batch_size, remainder.num_rows() - batch_size); - } - } else { - while remainder.num_rows() > 0 { - let batch_size = rng.gen_range(0..remainder.num_rows() + 1); - batches.push(remainder.slice(0, batch_size)); - remainder = - remainder.slice(batch_size, remainder.num_rows() - batch_size); - } - } - add_empty_batches(batches, &mut rng) - } - - #[tokio::test] - #[ignore] - async fn test1() { - let n_trials = vec![10]; - let n_rows = vec![100, 1000, 100_000]; - let distincts = vec![10, 1000, 100_000, 100_000_000]; - let n_batches = vec![10, 100_000_000]; - // let n_trials = vec![10]; - // let n_rows = vec![100_000]; - // let distincts = vec![100_000_000]; - // let n_batches = vec![100_000_000]; - for (n_trial, n_row, distinct, n_batch) in - iproduct!(n_trials, n_rows, distincts, n_batches) - { - let mut elapsed = vec![]; - for i in 0..n_trial { - let res = run_test(i, n_row, distinct, n_batch).await.unwrap(); - elapsed.push(res); - } - elapsed.sort(); - let tot_dur: Duration = elapsed.iter().sum(); - println!("------------------------------------"); - println!( - "n_row: {:?}, distinct: {:?}, n_batch: {:?}", - n_row, distinct, n_batch - ); - println!("elapsed mean: {:?}", tot_dur / elapsed.len() as u32); - println!("elapsed median: {:?}", elapsed[(n_trial / 2) as usize]); - println!("------------------------------------"); - } - } - - async fn run_test( - random_seed: u64, - n_row: usize, - distinct: usize, - n_batch: usize, - ) -> Result { - let in_data = - make_staggered_batches::(n_row, distinct, random_seed, n_batch); - let schema = in_data[0].schema(); - // let in_data = vec![concat_batches(&schema, &in_data).unwrap()]; - let in_exec = - Arc::new(MemoryExec::try_new(&[in_data], schema.clone(), None).unwrap()); - // print_batches(&in_data)?; - // let mut partitionby_exprs = vec![]; - // partitionby_exprs.push(col("a", &schema).unwrap()); - // let mut group_by_expr: Vec<(Arc, String)> = vec![]; - // group_by_expr.push((col("a", &schema).unwrap(), "dummy".to_string())); - // let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); - - let grouping_set = PhysicalGroupBy::new( - vec![(col("a", &schema)?, "a".to_string())], - vec![], - vec![vec![false]], - ); - - let aggregates: Vec> = vec![ - Arc::new(Sum::new( - col("x", &schema)?, - "Sum(x)".to_string(), - DataType::Int64, - )), - Arc::new(Min::new( - col("x", &schema)?, - "Min(x)".to_string(), - DataType::Int32, - )), - Arc::new(Max::new( - col("x", &schema)?, - "Max(x)".to_string(), - DataType::Int32, - )), - Arc::new(Avg::new( - col("x", &schema)?, - "Avg(x)".to_string(), - DataType::Float64, - )), - Arc::new(Sum::new( - col("y", &schema)?, - "Sum(y)".to_string(), - DataType::Int64, - )), - Arc::new(Min::new( - col("y", &schema)?, - "Min(y)".to_string(), - DataType::Int32, - )), - Arc::new(Max::new( - col("y", &schema)?, - "Max(y)".to_string(), - DataType::Int32, - )), - Arc::new(Avg::new( - col("y", &schema)?, - "Avg(y)".to_string(), - DataType::Float64, - )), - ]; - - let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - grouping_set, - aggregates, - in_exec, - schema, - )?) as _; - - let session_config = SessionConfig::new().with_batch_size(50); - let ctx = SessionContext::with_config(session_config); - let task_ctx = ctx.task_ctx(); - let now = Instant::now(); - let _collected_running = collect(aggregate_exec, task_ctx.clone()).await.unwrap(); - let elapsed = now.elapsed(); - // println!("Elapsed: {:.2?}", now.elapsed()); - // println!("Elapsed: {:.2?}", now_start.elapsed()); - // print_batches(&collected_running)?; - Ok(elapsed) - } -} From f7f80a638af19b47142a62eac8cb193b9390fd9b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 10 Jan 2023 11:08:34 +0300 Subject: [PATCH 05/19] minor changes --- datafusion/core/src/dataframe.rs | 2 -- 1 file changed, 2 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 82409471190e9..e9773dbdf3728 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -838,7 +838,6 @@ mod tests { use arrow::array::Int32Array; use arrow::datatypes::DataType; - use arrow::util::pretty::print_batches; use datafusion_expr::{ avg, cast, count, count_distinct, create_udf, expr, lit, max, min, sum, @@ -950,7 +949,6 @@ mod tests { ]; let df: Vec = df.aggregate(group_expr, aggr_expr)?.collect().await?; - print_batches(&df)?; assert_batches_sorted_eq!( vec![ From 6dc9ccfe05040a7abeaca84c44f24076d1631c11 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 10 Jan 2023 14:27:26 +0300 Subject: [PATCH 06/19] add test --- datafusion/core/tests/sql/window.rs | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 1167d57a4ffb2..ae6a89c3ce621 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2631,3 +2631,32 @@ mod tests { Ok(()) } } + +#[tokio::test] +async fn test_accumulator_row_accumulator() -> Result<()> { + let config = SessionConfig::new(); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + + let sql = "SELECT c1, c2, MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 + FROM aggregate_test_100 + GROUP BY c1, c2 + ORDER BY c1, c2 + LIMIT 5"; + + let actual = execute_to_batches(&ctx, sql).await; + let expected = vec![ + "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", + "| c1 | c2 | min1 | min2 | max1 | max2 | avg1 | min3 | cnt1 | sum1 |", + "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", + "| a | 1 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 774637006 | waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs | 4015442341 | 2437927011 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 5 | 6094771121.5 |", + "| a | 2 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 145294611 | ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 | 3717551163 | 2267588664 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 3 | 3401364777 |", + "| a | 3 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 431948861 | oLZ21P2JEDooxV1pU31cIxQHEeeoLu | 3998790955 | 2225685115.1666665 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 6 | 6676994872.5 |", + "| a | 4 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 466439833 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 2502326480 | 1655431654 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 4 | 3310812222.5 |", + "| a | 5 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 141047417 | QJYm7YRA3YetcBHI5wkMZeLXVmfuNy | 2496054700 | 1216992989.6666667 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 3 | 1825431770 |", + "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", + ]; + assert_batches_eq!(expected, &actual); + + Ok(()) +} From fd7b97415d2d3978032152ef39472cda9a8a2c83 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Jan 2023 13:36:22 +0300 Subject: [PATCH 07/19] remove row support for no grouping --- .../core/src/physical_plan/aggregates/mod.rs | 70 +-------- .../physical_plan/aggregates/no_grouping.rs | 143 ++---------------- datafusion/core/tests/sql/window.rs | 23 +++ 3 files changed, 38 insertions(+), 198 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index c168101236b60..805252e5b568b 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -29,7 +29,7 @@ use crate::physical_plan::{ use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{DataFusionError, Result}; use datafusion_expr::Accumulator; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ @@ -43,17 +43,13 @@ use std::sync::Arc; mod no_grouping; mod row_hash; -use crate::physical_plan::aggregates::row_hash::{ - read_as_batch, GroupedHashAggregateStreamV2, RowAggregationState, -}; +use crate::physical_plan::aggregates::row_hash::GroupedHashAggregateStreamV2; use crate::physical_plan::EquivalenceProperties; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::row_accumulator::RowAccumulator; use datafusion_physical_expr::equivalence::project_equivalence_properties; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::normalize_out_expr_with_alias_schema; -use datafusion_row::accessor::RowAccessor; -use datafusion_row::RowType; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -619,14 +615,9 @@ fn create_accumulators_v2( /// final value (mode = Final) or states (mode = Partial) fn finalize_aggregation( accumulators: &[AccumulatorItem], - row_accumulators: &[AccumulatorItemV2], mode: &AggregateMode, - aggr_schema: &Schema, - aggr_state: &mut RowAggregationState, - output_schema: &Schema, - indices: &[Vec<(usize, (usize, usize))>], ) -> datafusion_common::Result> { - let acc_res = match mode { + match mode { AggregateMode::Partial => { // build the vector of states let a = accumulators @@ -647,62 +638,7 @@ fn finalize_aggregation( .map(|accumulator| accumulator.evaluate().map(|v| v.to_array())) .collect::>>() } - }?; - - let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); - - let mut state_buffers = vec![aggr_state.group_states[0].aggregation_buffer.clone()]; - let mut columns: Vec = vec![]; - match mode { - AggregateMode::Partial => columns.extend(read_as_batch( - &state_buffers, - aggr_schema, - RowType::WordAligned, - )), - AggregateMode::Final | AggregateMode::FinalPartitioned => { - let mut results: Vec> = vec![vec![]; row_accumulators.len()]; - for buffer in state_buffers.iter_mut() { - state_accessor.point_to(0, buffer); - for (i, acc) in row_accumulators.iter().enumerate() { - results[i].push(acc.evaluate(&state_accessor).unwrap()); - } - } - // We skip over the first `columns.len()` elements. - // - // This shouldn't panic if the `output_schema` has enough fields. - let remaining_field_iterator = output_schema.fields()[columns.len()..].iter(); - - for (scalars, field) in results.into_iter().zip(remaining_field_iterator) { - if !scalars.is_empty() { - columns.push(ScalarValue::iter_to_array(scalars)?); - } else { - columns.push(arrow::array::new_empty_array(field.data_type())) - } - } - } - }; - let empty_arr = ScalarValue::iter_to_array(vec![ScalarValue::Null])?; - let n_res = indices[0] - .iter() - .map(|(_, range)| range.1 - range.0) - .sum::() - + indices[1] - .iter() - .map(|(_, range)| range.1 - range.0) - .sum::(); - let mut res = vec![empty_arr; n_res]; - let results = vec![acc_res, columns]; - for (outer, cur_res) in results.into_iter().enumerate() { - let mut start_idx = 0; - let cur_indices = &indices[outer]; - for (_idx, range) in cur_indices.iter() { - for elem in res.iter_mut().take(range.1).skip(range.0) { - *elem = cur_res[start_idx].clone(); - start_idx += 1; - } - } } - Ok(res) } /// Evaluates expressions against a record batch. diff --git a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs index 1b0a104c576f9..4d9de842f6f08 100644 --- a/datafusion/core/src/physical_plan/aggregates/no_grouping.rs +++ b/datafusion/core/src/physical_plan/aggregates/no_grouping.rs @@ -19,8 +19,8 @@ use crate::execution::context::TaskContext; use crate::physical_plan::aggregates::{ - aggregate_expressions, create_accumulators, create_accumulators_v2, evaluate_many, - finalize_aggregation, AccumulatorItem, AccumulatorItemV2, AggregateMode, + aggregate_expressions, create_accumulators, finalize_aggregation, AccumulatorItem, + AggregateMode, }; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; @@ -34,14 +34,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use crate::physical_plan::aggregates::row_hash::{ - aggr_state_schema, RowAggregationState, RowGroupState, -}; -use datafusion_row::accessor::RowAccessor; -use datafusion_row::layout::RowLayout; -use datafusion_row::RowType; use futures::stream::{Stream, StreamExt}; -use hashbrown::raw::RawTable; /// stream struct for aggregation without grouping columns pub(crate) struct AggregateStream { @@ -60,15 +53,9 @@ struct AggregateStreamInner { input: SendableRecordBatchStream, baseline_metrics: BaselineMetrics, aggregate_expressions: Vec>>, - row_aggregate_expressions: Vec>>, accumulators: Vec, - row_accumulators: Vec, reservation: MemoryReservation, finished: bool, - row_aggr_state: RowAggregationState, - row_aggr_layout: Arc, - row_aggr_schema: SchemaRef, - indices: Vec>, } impl AggregateStream { @@ -82,65 +69,8 @@ impl AggregateStream { context: Arc, partition: usize, ) -> Result { - let mut row_agg_indices = vec![]; - let mut normal_agg_indices = vec![]; - let mut start_idx = 0; - for (idx, expr) in aggr_expr.iter().enumerate() { - let n_field = match mode { - AggregateMode::Partial => expr.state_fields()?.len(), - _ => 1, - }; - if expr.row_accumulator_supported() { - row_agg_indices.push((idx, (start_idx, start_idx + n_field))); - } else { - normal_agg_indices.push((idx, (start_idx, start_idx + n_field))); - } - start_idx += n_field; - } - let indices = vec![normal_agg_indices, row_agg_indices]; - let row_aggr_exprs = aggr_expr - .clone() - .into_iter() - .filter(|elem| elem.row_accumulator_supported()) - .collect::>(); - let normal_aggr_exprs = aggr_expr - .clone() - .into_iter() - .filter(|elem| !elem.row_accumulator_supported()) - .collect::>(); - - let row_aggr_schema = aggr_state_schema(&row_aggr_exprs)?; - - let row_aggr_layout = - Arc::new(RowLayout::new(&row_aggr_schema, RowType::WordAligned)); - let reservation = - MemoryConsumer::new(format!("GroupedHashAggregateStreamV2[{}]", partition)) - .register(context.memory_pool()); - - // Add new entry to group_states and save newly created index - let group_state = RowGroupState { - group_by_values: Box::new([]), - aggregation_buffer: vec![0; row_aggr_layout.fixed_part_width()], - accumulator_set: vec![], - indices: vec![0_u32], // 1.3 - }; - let row_aggr_state = RowAggregationState { - reservation, - map: RawTable::with_capacity(0), - group_states: vec![group_state], - }; - - let all_aggregate_expressions = aggregate_expressions(&aggr_expr, &mode, 0)?; - let mut normal_aggregate_expressions = vec![]; - for (idx, _) in &indices[0] { - normal_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) - } - let mut row_aggregate_expressions = vec![]; - for (idx, _) in &indices[1] { - row_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) - } - let normal_accumulators = create_accumulators(&normal_aggr_exprs)?; - let row_accumulators = create_accumulators_v2(&row_aggr_exprs)?; + let aggregate_expressions = aggregate_expressions(&aggr_expr, &mode, 0)?; + let accumulators = create_accumulators(&aggr_expr)?; let reservation = MemoryConsumer::new(format!("AggregateStream[{partition}]")) .register(context.memory_pool()); @@ -150,16 +80,10 @@ impl AggregateStream { mode, input, baseline_metrics, - aggregate_expressions: normal_aggregate_expressions, - row_aggregate_expressions, - accumulators: normal_accumulators, - row_accumulators, + aggregate_expressions, + accumulators, reservation, finished: false, - row_aggr_state, - row_aggr_layout, - row_aggr_schema, - indices, }; let stream = futures::stream::unfold(inner, |mut this| async move { if this.finished { @@ -176,11 +100,7 @@ impl AggregateStream { &this.mode, &batch, &mut this.accumulators, - &mut this.row_accumulators, &this.aggregate_expressions, - &this.row_aggregate_expressions, - &mut this.row_aggr_state, - this.row_aggr_layout.clone(), ); timer.done(); @@ -199,20 +119,12 @@ impl AggregateStream { None => { this.finished = true; let timer = this.baseline_metrics.elapsed_compute().timer(); - let result = finalize_aggregation( - &this.accumulators, - &this.row_accumulators, - &this.mode, - &this.row_aggr_schema, - &mut this.row_aggr_state, - &this.schema, - &this.indices, - ) - .map_err(|e| ArrowError::ExternalError(Box::new(e))) - .and_then(|columns| { - RecordBatch::try_new(this.schema.clone(), columns) - }) - .record_output(&this.baseline_metrics); + let result = finalize_aggregation(&this.accumulators, &this.mode) + .map_err(|e| ArrowError::ExternalError(Box::new(e))) + .and_then(|columns| { + RecordBatch::try_new(this.schema.clone(), columns) + }) + .record_output(&this.baseline_metrics); timer.done(); @@ -256,43 +168,12 @@ impl RecordBatchStream for AggregateStream { /// If successfull, this returns the additional number of bytes that were allocated during this process. /// /// TODO: Make this a member function -#[allow(clippy::too_many_arguments)] fn aggregate_batch( mode: &AggregateMode, batch: &RecordBatch, accumulators: &mut [AccumulatorItem], - row_accumulators: &mut [AccumulatorItemV2], expressions: &[Vec>], - row_expressions: &[Vec>], - row_aggr_state: &mut RowAggregationState, - state_layout: Arc, ) -> Result { - let RowAggregationState { group_states, .. } = row_aggr_state; - let group_state = &mut group_states[0]; - let aggr_input_values = evaluate_many(row_expressions, batch)?; - row_accumulators - .iter_mut() - .zip(aggr_input_values.iter()) - .map(|(accumulator, aggr_array)| (accumulator, aggr_array)) - .try_for_each(|(accumulator, values)| { - let mut state_accessor = RowAccessor::new_from_layout(state_layout.clone()); - state_accessor.point_to(0, group_state.aggregation_buffer.as_mut_slice()); - match mode { - AggregateMode::Partial => { - accumulator.update_batch(values, &mut state_accessor) - } - AggregateMode::FinalPartitioned | AggregateMode::Final => { - // note: the aggregation here is over states, not values, thus the merge - accumulator.merge_batch(values, &mut state_accessor) - } - } - }) - // 2.5 - .and({ - group_state.indices.clear(); - Ok(()) - })?; - let mut allocated = 0usize; // 1.1 iterate accumulators and respective expressions together diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index ae6a89c3ce621..e88aa224a7127 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2660,3 +2660,26 @@ async fn test_accumulator_row_accumulator() -> Result<()> { Ok(()) } + +#[tokio::test] +async fn test_accumulator_row_accumulator_v2() -> Result<()> { + let config = SessionConfig::new(); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + + let sql = "SELECT MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 + FROM aggregate_test_100 + LIMIT 5"; + + let actual = execute_to_batches(&ctx, sql).await; + let expected = vec![ + "+--------------------------------+----------+--------------------------------+------------+--------------+--------------------------------+------+----------------+", + "| min1 | min2 | max1 | max2 | avg1 | min3 | cnt1 | sum1 |", + "+--------------------------------+----------+--------------------------------+------------+--------------+--------------------------------+------+----------------+", + "| 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 28774375 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 4268716378 | 2220897700.6 | 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 100 | 111043376209.5 |", + "+--------------------------------+----------+--------------------------------+------------+--------------+--------------------------------+------+----------------+", + ]; + assert_batches_eq!(expected, &actual); + + Ok(()) +} From 96f33f4b25dfc0c7fb986233756efcf3d30ae597 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Jan 2023 16:54:13 +0300 Subject: [PATCH 08/19] fix ci bug --- .../src/physical_plan/aggregates/row_hash.rs | 39 ++++++++----------- 1 file changed, 17 insertions(+), 22 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 2b1411b865e5a..69969c92c684d 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -17,6 +17,7 @@ //! Hash aggregation through row format +use std::cmp::min; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; @@ -629,15 +630,13 @@ fn create_batch_from_map( )))); } - let group_buffers = row_aggr_state - .group_states + let end_idx = min(skip_items + batch_size, row_aggr_state.group_states.len()); + let group_state_chunk = &row_aggr_state.group_states[skip_items..end_idx]; + let group_buffers = group_state_chunk .iter() - .skip(skip_items) - .take(batch_size) .map(|gs| (gs.group_by_values.clone())) .collect::>(); - // let mut group_by_columns = read_as_batch(&group_buffers, group_schema, RowType::Compact)?; let n_row = group_buffers.len(); if n_row == 0 { return Ok(Some(RecordBatch::new_empty(Arc::new( @@ -645,7 +644,7 @@ fn create_batch_from_map( )))); } // First, output all group by exprs - let group_by_columns = (0..num_group_expr) + let all_group_by_columns = (0..num_group_expr) .map(|idx| { ScalarValue::iter_to_array(group_buffers.iter().map(|x| x[idx].clone())) }) @@ -653,11 +652,8 @@ fn create_batch_from_map( let mut row_columns = vec![]; let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); - let mut state_buffers = row_aggr_state - .group_states + let mut state_buffers = group_state_chunk .iter() - .skip(skip_items) - .take(batch_size) .map(|gs| gs.aggregation_buffer.clone()) .collect::>(); @@ -679,7 +675,7 @@ fn create_batch_from_map( // // This shouldn't panic if the `output_schema` has enough fields. let remaining_field_iterator = - output_schema.fields()[group_by_columns.len()..].iter(); + output_schema.fields()[all_group_by_columns.len()..].iter(); for (scalars, field) in results.into_iter().zip(remaining_field_iterator) { if !scalars.is_empty() { @@ -730,25 +726,25 @@ fn create_batch_from_map( for y in 0..state_len { match mode { AggregateMode::Partial => { - let res = ScalarValue::iter_to_array( - row_aggr_state.group_states.iter().map(|row_group_state| { + let res = ScalarValue::iter_to_array(group_state_chunk.iter().map( + |row_group_state| { row_group_state.accumulator_set[x] .state() .map(|x| x[y].clone()) .expect("unexpected accumulator state in hash aggregate") - }), - )?; + }, + ))?; columns.push(res); } AggregateMode::Final | AggregateMode::FinalPartitioned => { - let res = ScalarValue::iter_to_array( - row_aggr_state.group_states.iter().map(|row_group_state| { + let res = ScalarValue::iter_to_array(group_state_chunk.iter().map( + |row_group_state| { row_group_state.accumulator_set[x] .evaluate() .expect("unexpected accumulator state in hash aggregate") - }), - )?; + }, + ))?; columns.push(res); } } @@ -779,9 +775,9 @@ fn create_batch_from_map( .iter() .map(|(_, range)| range.1 - range.0) .sum::() - + group_by_columns.len(); + + all_group_by_columns.len(); let mut res = vec![empty_arr; n_res]; - for (idx, column) in group_by_columns.into_iter().enumerate() { + for (idx, column) in all_group_by_columns.into_iter().enumerate() { res[idx] = column; } @@ -796,7 +792,6 @@ fn create_batch_from_map( } } } - Ok(Some(RecordBatch::try_new( Arc::new(output_schema.to_owned()), res, From a87ca9b5b5457a42ffa491fa1cab673567f06bf7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Jan 2023 17:10:50 +0300 Subject: [PATCH 09/19] minor changes, rename --- .../core/src/physical_plan/aggregates/mod.rs | 22 +++++------- .../src/physical_plan/aggregates/row_hash.rs | 35 +++++++++---------- 2 files changed, 25 insertions(+), 32 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 805252e5b568b..3ab4312dacc91 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -43,7 +43,7 @@ use std::sync::Arc; mod no_grouping; mod row_hash; -use crate::physical_plan::aggregates::row_hash::GroupedHashAggregateStreamV2; +use crate::physical_plan::aggregates::row_hash::GroupedHashAggregateStream; use crate::physical_plan::EquivalenceProperties; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::row_accumulator::RowAccumulator; @@ -149,14 +149,14 @@ impl PhysicalGroupBy { enum StreamType { AggregateStream(AggregateStream), - GroupedHashAggregateStreamV2(GroupedHashAggregateStreamV2), + GroupedHashAggregateStream(GroupedHashAggregateStream), } impl From for SendableRecordBatchStream { fn from(stream: StreamType) -> Self { match stream { StreamType::AggregateStream(stream) => Box::pin(stream), - StreamType::GroupedHashAggregateStreamV2(stream) => Box::pin(stream), + StreamType::GroupedHashAggregateStream(stream) => Box::pin(stream), } } } @@ -286,8 +286,8 @@ impl AggregateExec { partition, )?)) } else { - Ok(StreamType::GroupedHashAggregateStreamV2( - GroupedHashAggregateStreamV2::new( + Ok(StreamType::GroupedHashAggregateStream( + GroupedHashAggregateStream::new( self.mode, self.schema.clone(), self.group_by.clone(), @@ -597,7 +597,7 @@ fn create_accumulators( .collect::>>() } -fn create_accumulators_v2( +fn create_row_accumulators( aggr_expr: &[Arc], ) -> datafusion_common::Result> { let mut state_index = 0; @@ -1163,16 +1163,10 @@ mod tests { assert!(matches!(stream, StreamType::AggregateStream(_))); } 1 => { - assert!(matches!( - stream, - StreamType::GroupedHashAggregateStreamV2(_) - )); + assert!(matches!(stream, StreamType::GroupedHashAggregateStream(_))); } 2 => { - assert!(matches!( - stream, - StreamType::GroupedHashAggregateStreamV2(_) - )); + assert!(matches!(stream, StreamType::GroupedHashAggregateStream(_))); } _ => panic!("Unknown version: {version}"), } diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 69969c92c684d..6db33397f2ebc 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -71,7 +71,7 @@ use hashbrown::raw::RawTable; /// /// [Compact]: datafusion_row::layout::RowType::Compact /// [WordAligned]: datafusion_row::layout::RowType::WordAligned -pub(crate) struct GroupedHashAggregateStreamV2 { +pub(crate) struct GroupedHashAggregateStream { stream: BoxStream<'static, ArrowResult>, schema: SchemaRef, } @@ -81,7 +81,7 @@ pub(crate) struct GroupedHashAggregateStreamV2 { /// This is wrapped into yet another struct because we need to interact with the async memory management subsystem /// during poll. To have as little code "weirdness" as possible, we chose to just use [`BoxStream`] together with /// [`futures::stream::unfold`]. The latter requires a state object, which is [`GroupedHashAggregateStreamV2Inner`]. -struct GroupedHashAggregateStreamV2Inner { +struct GroupedHashAggregateStreamInner { schema: SchemaRef, input: SendableRecordBatchStream, mode: AggregateMode, @@ -106,7 +106,7 @@ struct GroupedHashAggregateStreamV2Inner { indices: Vec>, } -pub fn aggr_state_schema(aggr_expr: &[Arc]) -> Result { +fn aggr_state_schema(aggr_expr: &[Arc]) -> Result { let fields = aggr_expr .iter() .flat_map(|expr| expr.state_fields().unwrap().into_iter()) @@ -118,7 +118,7 @@ fn is_supported(elem: &Arc, group_schema: &Schema) -> bool { elem.row_accumulator_supported() && row_supported(group_schema, RowType::Compact) } -impl GroupedHashAggregateStreamV2 { +impl GroupedHashAggregateStream { /// Create a new GroupedRowHashAggregateStream #[allow(clippy::too_many_arguments)] pub fn new( @@ -152,12 +152,12 @@ impl GroupedHashAggregateStreamV2 { } let indices = vec![normal_agg_indices, row_agg_indices]; - let row_aggr_exprs = aggr_expr + let row_aggr_expr = aggr_expr .clone() .into_iter() .filter(|elem| is_supported(elem, &group_schema)) .collect::>(); - let normal_aggr_exprs = aggr_expr + let normal_aggr_expr = aggr_expr .clone() .into_iter() .filter(|elem| !is_supported(elem, &group_schema)) @@ -176,15 +176,14 @@ impl GroupedHashAggregateStreamV2 { row_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) } - // let accumulators = aggregates::create_accumulators_v2(&aggr_expr)?; - let row_accumulators = aggregates::create_accumulators_v2(&row_aggr_exprs)?; + let row_accumulators = aggregates::create_row_accumulators(&row_aggr_expr)?; - let row_aggr_schema = aggr_state_schema(&row_aggr_exprs)?; + let row_aggr_schema = aggr_state_schema(&row_aggr_expr)?; let row_aggr_layout = Arc::new(RowLayout::new(&row_aggr_schema, RowType::WordAligned)); let row_reservation = - MemoryConsumer::new(format!("GroupedHashAggregateStreamV2[{}]", partition)) + MemoryConsumer::new(format!("GroupedHashAggregateStream[{}]", partition)) .register(context.memory_pool()); let row_aggr_state = RowAggregationState { @@ -195,12 +194,12 @@ impl GroupedHashAggregateStreamV2 { timer.done(); - let inner = GroupedHashAggregateStreamV2Inner { + let inner = GroupedHashAggregateStreamInner { schema: Arc::clone(&schema), mode, input, group_by, - normal_aggr_expr: normal_aggr_exprs, + normal_aggr_expr, row_accumulators, row_aggr_schema, row_aggr_layout, @@ -289,7 +288,7 @@ impl GroupedHashAggregateStreamV2 { } } -impl Stream for GroupedHashAggregateStreamV2 { +impl Stream for GroupedHashAggregateStream { type Item = ArrowResult; fn poll_next( @@ -301,7 +300,7 @@ impl Stream for GroupedHashAggregateStreamV2 { } } -impl RecordBatchStream for GroupedHashAggregateStreamV2 { +impl RecordBatchStream for GroupedHashAggregateStream { fn schema(&self) -> SchemaRef { self.schema.clone() } @@ -644,7 +643,7 @@ fn create_batch_from_map( )))); } // First, output all group by exprs - let all_group_by_columns = (0..num_group_expr) + let group_by_columns = (0..num_group_expr) .map(|idx| { ScalarValue::iter_to_array(group_buffers.iter().map(|x| x[idx].clone())) }) @@ -675,7 +674,7 @@ fn create_batch_from_map( // // This shouldn't panic if the `output_schema` has enough fields. let remaining_field_iterator = - output_schema.fields()[all_group_by_columns.len()..].iter(); + output_schema.fields()[group_by_columns.len()..].iter(); for (scalars, field) in results.into_iter().zip(remaining_field_iterator) { if !scalars.is_empty() { @@ -775,9 +774,9 @@ fn create_batch_from_map( .iter() .map(|(_, range)| range.1 - range.0) .sum::() - + all_group_by_columns.len(); + + group_by_columns.len(); let mut res = vec![empty_arr; n_res]; - for (idx, column) in all_group_by_columns.into_iter().enumerate() { + for (idx, column) in group_by_columns.into_iter().enumerate() { res[idx] = column; } From a20ab45008b3b7be129d335efe70304f18a654cc Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 11 Jan 2023 22:25:44 -0600 Subject: [PATCH 10/19] Refactor Part 1 --- .../src/physical_plan/aggregates/row_hash.rs | 255 ++++++++---------- 1 file changed, 107 insertions(+), 148 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 6db33397f2ebc..ffb58de8e427a 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -38,14 +38,14 @@ use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use arrow::compute::cast; -use arrow::datatypes::Schema; -use arrow::{array::ArrayRef, compute}; use arrow::{ - array::{Array, UInt32Builder}, + array::{new_null_array, Array, ArrayRef, UInt32Builder}, + compute, + compute::cast, + datatypes::{DataType, Schema, SchemaRef}, error::{ArrowError, Result as ArrowResult}, + record_batch::RecordBatch, }; -use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_expr::Accumulator; use datafusion_physical_expr::hash_utils::create_hashes; @@ -76,11 +76,11 @@ pub(crate) struct GroupedHashAggregateStream { schema: SchemaRef, } -/// Actual implementation of [`GroupedHashAggregateStreamV2`]. +/// Actual implementation of [`GroupedHashAggregateStream`]. /// /// This is wrapped into yet another struct because we need to interact with the async memory management subsystem /// during poll. To have as little code "weirdness" as possible, we chose to just use [`BoxStream`] together with -/// [`futures::stream::unfold`]. The latter requires a state object, which is [`GroupedHashAggregateStreamV2Inner`]. +/// [`futures::stream::unfold`]. The latter requires a state object, which is [`GroupedHashAggregateStreamInner`]. struct GroupedHashAggregateStreamInner { schema: SchemaRef, input: SendableRecordBatchStream, @@ -103,7 +103,7 @@ struct GroupedHashAggregateStreamInner { /// if the result is chunked into batches, /// last offset is preserved for continuation. row_group_skip_position: usize, - indices: Vec>, + indices: [Vec<(usize, usize, usize)>; 2], } fn aggr_state_schema(aggr_expr: &[Arc]) -> Result { @@ -114,12 +114,12 @@ fn aggr_state_schema(aggr_expr: &[Arc]) -> Result Ok(Arc::new(Schema::new(fields))) } -fn is_supported(elem: &Arc, group_schema: &Schema) -> bool { - elem.row_accumulator_supported() && row_supported(group_schema, RowType::Compact) +fn is_supported(expr: &Arc, group_schema: &Schema) -> bool { + expr.row_accumulator_supported() && row_supported(group_schema, RowType::Compact) } impl GroupedHashAggregateStream { - /// Create a new GroupedRowHashAggregateStream + /// Create a new GroupedHashAggregateStream #[allow(clippy::too_many_arguments)] pub fn new( mode: AggregateMode, @@ -135,45 +135,37 @@ impl GroupedHashAggregateStream { let timer = baseline_metrics.elapsed_compute().timer(); let group_schema = group_schema(&schema, group_by.expr.len()); + let mut start_idx = group_by.expr.len(); + let mut row_aggr_expr = vec![]; let mut row_agg_indices = vec![]; + let mut row_aggregate_expressions = vec![]; + let mut normal_aggr_expr = vec![]; let mut normal_agg_indices = vec![]; - let mut start_idx = group_by.expr.len(); - for (idx, expr) in aggr_expr.iter().enumerate() { - let n_field = match mode { + let mut normal_aggregate_expressions = vec![]; + // The expressions to evaluate the batch, one vec of expressions per aggregation. + // Assume create_schema() always put group columns in front of aggr columns, we set + // col_idx_base to group expression count. + let all_aggregate_expressions = + aggregates::aggregate_expressions(&aggr_expr, &mode, start_idx)?; + for (idx, (expr, others)) in aggr_expr + .iter() + .zip(all_aggregate_expressions.into_iter()) + .enumerate() + { + let n_fields = match mode { AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; if is_supported(expr, &group_schema) { - row_agg_indices.push((idx, (start_idx, start_idx + n_field))); + row_aggregate_expressions.push(others); + row_agg_indices.push((idx, start_idx, start_idx + n_fields)); + row_aggr_expr.push(expr.clone()); } else { - normal_agg_indices.push((idx, (start_idx, start_idx + n_field))); + normal_aggregate_expressions.push(others); + normal_agg_indices.push((idx, start_idx, start_idx + n_fields)); + normal_aggr_expr.push(expr.clone()); } - start_idx += n_field; - } - let indices = vec![normal_agg_indices, row_agg_indices]; - - let row_aggr_expr = aggr_expr - .clone() - .into_iter() - .filter(|elem| is_supported(elem, &group_schema)) - .collect::>(); - let normal_aggr_expr = aggr_expr - .clone() - .into_iter() - .filter(|elem| !is_supported(elem, &group_schema)) - .collect::>(); - // The expressions to evaluate the batch, one vec of expressions per aggregation. - // Assume create_schema() always put group columns in front of aggr columns, we set - // col_idx_base to group expression count. - let all_aggregate_expressions = - aggregates::aggregate_expressions(&aggr_expr, &mode, group_by.expr.len())?; - let mut normal_aggregate_expressions = vec![]; - for (idx, _) in &indices[0] { - normal_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) - } - let mut row_aggregate_expressions = vec![]; - for (idx, _) in &indices[1] { - row_aggregate_expressions.push(all_aggregate_expressions[*idx].clone()) + start_idx += n_fields; } let row_accumulators = aggregates::create_row_accumulators(&row_aggr_expr)?; @@ -182,12 +174,10 @@ impl GroupedHashAggregateStream { let row_aggr_layout = Arc::new(RowLayout::new(&row_aggr_schema, RowType::WordAligned)); - let row_reservation = - MemoryConsumer::new(format!("GroupedHashAggregateStream[{}]", partition)) - .register(context.memory_pool()); + let name = format!("GroupedHashAggregateStream[{}]", partition); let row_aggr_state = RowAggregationState { - reservation: row_reservation, + reservation: MemoryConsumer::new(name).register(context.memory_pool()), map: RawTable::with_capacity(0), group_states: Vec::with_capacity(0), }; @@ -210,7 +200,7 @@ impl GroupedHashAggregateStream { random_state: Default::default(), batch_size, row_group_skip_position: 0, - indices, + indices: [normal_agg_indices, row_agg_indices], }; let stream = futures::stream::unfold(inner, |mut this| async move { @@ -310,7 +300,7 @@ impl RecordBatchStream for GroupedHashAggregateStream { /// /// If successfull, this returns the additional number of bytes that were allocated during this process. /// -/// TODO: Make this a member function of [`GroupedHashAggregateStreamV2`] +/// TODO: Make this a member function of [`GroupedHashAggregateStream`] #[allow(clippy::too_many_arguments)] fn group_aggregate_batch( mode: &AggregateMode, @@ -617,38 +607,24 @@ fn create_batch_from_map( row_aggr_state: &mut RowAggregationState, row_accumulators: &mut [AccumulatorItemV2], output_schema: &Schema, - indices: &[Vec<(usize, (usize, usize))>], + indices: &[Vec<(usize, usize, usize)>], num_group_expr: usize, ) -> ArrowResult> { if skip_items > row_aggr_state.group_states.len() { return Ok(None); } if row_aggr_state.group_states.is_empty() { - return Ok(Some(RecordBatch::new_empty(Arc::new( - output_schema.to_owned(), - )))); + let schema = Arc::new(output_schema.to_owned()); + return Ok(Some(RecordBatch::new_empty(schema))); } let end_idx = min(skip_items + batch_size, row_aggr_state.group_states.len()); let group_state_chunk = &row_aggr_state.group_states[skip_items..end_idx]; - let group_buffers = group_state_chunk - .iter() - .map(|gs| (gs.group_by_values.clone())) - .collect::>(); - let n_row = group_buffers.len(); - if n_row == 0 { - return Ok(Some(RecordBatch::new_empty(Arc::new( - output_schema.to_owned(), - )))); + if group_state_chunk.is_empty() { + let schema = Arc::new(output_schema.to_owned()); + return Ok(Some(RecordBatch::new_empty(schema))); } - // First, output all group by exprs - let group_by_columns = (0..num_group_expr) - .map(|idx| { - ScalarValue::iter_to_array(group_buffers.iter().map(|x| x[idx].clone())) - }) - .collect::>>()?; - let mut row_columns = vec![]; let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); let mut state_buffers = group_state_chunk @@ -656,136 +632,119 @@ fn create_batch_from_map( .map(|gs| gs.aggregation_buffer.clone()) .collect::>(); - match mode { - AggregateMode::Partial => row_columns.extend(read_as_batch( - &state_buffers, - aggr_schema, - RowType::WordAligned, - )), + let output_fields = output_schema.fields(); + let mut row_columns = match mode { + AggregateMode::Partial => { + read_as_batch(&state_buffers, aggr_schema, RowType::WordAligned) + } AggregateMode::Final | AggregateMode::FinalPartitioned => { let mut results: Vec> = vec![vec![]; row_accumulators.len()]; for buffer in state_buffers.iter_mut() { state_accessor.point_to(0, buffer); for (i, acc) in row_accumulators.iter().enumerate() { - results[i].push(acc.evaluate(&state_accessor).unwrap()); + results[i].push(acc.evaluate(&state_accessor)?); } } // We skip over the first `columns.len()` elements. // // This shouldn't panic if the `output_schema` has enough fields. - let remaining_field_iterator = - output_schema.fields()[group_by_columns.len()..].iter(); - - for (scalars, field) in results.into_iter().zip(remaining_field_iterator) { - if !scalars.is_empty() { - row_columns.push(ScalarValue::iter_to_array(scalars)?); - } else { - row_columns.push(arrow::array::new_empty_array(field.data_type())) - } - } + let remaining_fields = output_fields[num_group_expr..].iter(); + results + .into_iter() + .zip(remaining_fields) + .map(|(scalars, field)| { + if scalars.is_empty() { + Ok(arrow::array::new_empty_array(field.data_type())) + } else { + ScalarValue::iter_to_array(scalars) + } + }) + .collect::>>()? } - } + }; // cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output - let row_column_indices = indices[1].clone(); let mut start_idx = 0; - let mut new_row_columns = vec![]; - for (_row_column_idx, range) in row_column_indices.iter() { - for idx in range.0..range.1 { - let desired_datatype = output_schema.fields()[idx].data_type(); - new_row_columns.push(cast(&row_columns[start_idx], desired_datatype)?); + for (_, begin, end) in indices[1].iter() { + for field in &output_fields[*begin..*end] { + row_columns[start_idx] = cast(&row_columns[start_idx], field.data_type())?; start_idx += 1; } } - let row_columns = new_row_columns; - // RecordBatch::try_new(Arc::new(output_schema.to_owned()), row_columns).map(Some) - - let mut columns = vec![]; - let accs = &row_aggr_state.group_states[0].accumulator_set; - let mut acc_data_types: Vec = vec![]; - // Calculate number/shape of state arrays - match mode { - AggregateMode::Partial => { - for acc in accs.iter() { - let state = acc.state()?; - acc_data_types.push(state.len()); - } - } + let accs = &row_aggr_state.group_states[0].accumulator_set; + let acc_data_types = match mode { + AggregateMode::Partial => accs + .iter() + .map(|a| a.state().map(|s| s.len())) + .collect::>>()?, AggregateMode::Final | AggregateMode::FinalPartitioned => { - acc_data_types = vec![1; accs.len()]; + vec![1; accs.len()] } - } + }; // next, output aggregates: either intermediate state or final output + let mut columns = vec![]; for (x, &state_len) in acc_data_types.iter().enumerate() { for y in 0..state_len { - match mode { - AggregateMode::Partial => { - let res = ScalarValue::iter_to_array(group_state_chunk.iter().map( - |row_group_state| { - row_group_state.accumulator_set[x] - .state() - .map(|x| x[y].clone()) - .expect("unexpected accumulator state in hash aggregate") - }, - ))?; - - columns.push(res); - } + columns.push(match mode { + AggregateMode::Partial => ScalarValue::iter_to_array( + group_state_chunk.iter().map(|row_group_state| { + row_group_state.accumulator_set[x] + .state() + .map(|x| x[y].clone()) + .expect("unexpected accumulator state in hash aggregate") + }), + ), AggregateMode::Final | AggregateMode::FinalPartitioned => { - let res = ScalarValue::iter_to_array(group_state_chunk.iter().map( + ScalarValue::iter_to_array(group_state_chunk.iter().map( |row_group_state| { row_group_state.accumulator_set[x] .evaluate() .expect("unexpected accumulator state in hash aggregate") }, - ))?; - columns.push(res); + )) } - } + }?); } } // cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output - let column_indices = indices[0].clone(); let mut start_idx = 0; - let mut new_columns = vec![]; - for (_column_idx, range) in column_indices.iter() { - for idx in range.0..range.1 { - let desired_datatype = output_schema.fields()[idx].data_type(); - new_columns.push(cast(&columns[start_idx], desired_datatype)?); + for (_, begin, end) in indices[0].iter() { + for field in &output_fields[*begin..*end] { + columns[start_idx] = cast(&columns[start_idx], field.data_type())?; start_idx += 1; } } - let columns = new_columns; - let empty_arr = ScalarValue::iter_to_array(vec![ScalarValue::Null])?; - let n_res = indices[0] + let mut res = (0..num_group_expr) + .map(|idx| { + ScalarValue::iter_to_array( + group_state_chunk + .iter() + .map(|gs| gs.group_by_values[idx].clone()), + ) + }) + .collect::>>()?; + let extra: usize = indices .iter() - .map(|(_, range)| range.1 - range.0) - .sum::() - + indices[1] - .iter() - .map(|(_, range)| range.1 - range.0) - .sum::() - + group_by_columns.len(); - let mut res = vec![empty_arr; n_res]; - for (idx, column) in group_by_columns.into_iter().enumerate() { - res[idx] = column; - } + .flatten() + .map(|(_, begin, end)| end - begin) + .sum(); + let empty_arr = new_null_array(&DataType::Null, 1); + res.extend(std::iter::repeat(empty_arr).take(extra)); let results = vec![columns, row_columns]; for (outer, cur_res) in results.into_iter().enumerate() { let mut start_idx = 0; - let cur_indices = &indices[outer]; - for (_idx, range) in cur_indices.iter() { - for elem in res.iter_mut().take(range.1).skip(range.0) { + for (_, begin, end) in indices[outer].iter() { + for elem in res.iter_mut().take(*end).skip(*begin) { *elem = cur_res[start_idx].clone(); start_idx += 1; } From 29b45c8b0ad0229af9266acaeb6ed26070b91fa8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Jan 2023 18:50:18 +0300 Subject: [PATCH 11/19] simplify code --- .../core/src/physical_plan/aggregates/mod.rs | 4 +- .../src/physical_plan/aggregates/row_hash.rs | 91 ++++++++++--------- 2 files changed, 51 insertions(+), 44 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/mod.rs b/datafusion/core/src/physical_plan/aggregates/mod.rs index 3ab4312dacc91..9a7fc7de48c39 100644 --- a/datafusion/core/src/physical_plan/aggregates/mod.rs +++ b/datafusion/core/src/physical_plan/aggregates/mod.rs @@ -586,7 +586,7 @@ fn merge_expressions( } pub(crate) type AccumulatorItem = Box; -pub(crate) type AccumulatorItemV2 = Box; +pub(crate) type RowAccumulatorItem = Box; fn create_accumulators( aggr_expr: &[Arc], @@ -599,7 +599,7 @@ fn create_accumulators( fn create_row_accumulators( aggr_expr: &[Arc], -) -> datafusion_common::Result> { +) -> datafusion_common::Result> { let mut state_index = 0; aggr_expr .iter() diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index ffb58de8e427a..b387e2beb6ca2 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -18,6 +18,7 @@ //! Hash aggregation through row format use std::cmp::min; +use std::collections::VecDeque; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; @@ -30,8 +31,8 @@ use crate::error::Result; use crate::execution::context::TaskContext; use crate::execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use crate::physical_plan::aggregates::{ - evaluate_group_by, evaluate_many, group_schema, AccumulatorItem, AccumulatorItemV2, - AggregateMode, PhysicalGroupBy, + evaluate_group_by, evaluate_many, group_schema, AccumulatorItem, AggregateMode, + PhysicalGroupBy, RowAccumulatorItem, }; use crate::physical_plan::metrics::{BaselineMetrics, RecordOutput}; use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; @@ -91,7 +92,7 @@ struct GroupedHashAggregateStreamInner { row_aggregate_expressions: Vec>>, group_by: PhysicalGroupBy, - row_accumulators: Vec, + row_accumulators: Vec, row_aggr_schema: SchemaRef, row_aggr_layout: Arc, @@ -307,7 +308,7 @@ fn group_aggregate_batch( random_state: &RandomState, grouping_set: &PhysicalGroupBy, normal_aggr_expr: &[Arc], - row_accumulators: &mut [AccumulatorItemV2], + row_accumulators: &mut [RowAccumulatorItem], state_layout: Arc, batch: RecordBatch, aggr_state: &mut RowAggregationState, @@ -605,7 +606,7 @@ fn create_batch_from_map( batch_size: usize, skip_items: usize, row_aggr_state: &mut RowAggregationState, - row_accumulators: &mut [AccumulatorItemV2], + row_accumulators: &mut [RowAccumulatorItem], output_schema: &Schema, indices: &[Vec<(usize, usize, usize)>], num_group_expr: usize, @@ -633,7 +634,7 @@ fn create_batch_from_map( .collect::>(); let output_fields = output_schema.fields(); - let mut row_columns = match mode { + let row_columns = match mode { AggregateMode::Partial => { read_as_batch(&state_buffers, aggr_schema, RowType::WordAligned) } @@ -645,10 +646,13 @@ fn create_batch_from_map( results[i].push(acc.evaluate(&state_accessor)?); } } - // We skip over the first `columns.len()` elements. - // - // This shouldn't panic if the `output_schema` has enough fields. - let remaining_fields = output_fields[num_group_expr..].iter(); + // We fill fields corresponding to row accumulators e.g indices[1] + let mut remaining_fields = vec![]; + for (_, begin, end) in indices[1].iter() { + for field in &output_fields[*begin..*end] { + remaining_fields.push(field); + } + } results .into_iter() .zip(remaining_fields) @@ -656,27 +660,21 @@ fn create_batch_from_map( if scalars.is_empty() { Ok(arrow::array::new_empty_array(field.data_type())) } else { - ScalarValue::iter_to_array(scalars) + let elem = ScalarValue::iter_to_array(scalars)?; + // cast output if needed (e.g. for types like Dictionary where + // the intermediate GroupByScalar type was not the same as the + // output + cast(&elem, field.data_type()) + .map_err(DataFusionError::ArrowError) } }) .collect::>>()? } }; - // cast output if needed (e.g. for types like Dictionary where - // the intermediate GroupByScalar type was not the same as the - // output - let mut start_idx = 0; - for (_, begin, end) in indices[1].iter() { - for field in &output_fields[*begin..*end] { - row_columns[start_idx] = cast(&row_columns[start_idx], field.data_type())?; - start_idx += 1; - } - } - // Calculate number/shape of state arrays let accs = &row_aggr_state.group_states[0].accumulator_set; - let acc_data_types = match mode { + let acc_state_lengths = match mode { AggregateMode::Partial => accs .iter() .map(|a| a.state().map(|s| s.len())) @@ -686,40 +684,49 @@ fn create_batch_from_map( } }; + // We fill fields corresponding to accumulators e.g indices[0] + let mut remaining_fields = VecDeque::new(); + for (_, begin, end) in indices[0].iter() { + for field in &output_fields[*begin..*end] { + remaining_fields.push_back(field); + } + } // next, output aggregates: either intermediate state or final output let mut columns = vec![]; - for (x, &state_len) in acc_data_types.iter().enumerate() { + for (idx, &state_len) in acc_state_lengths.iter().enumerate() { for y in 0..state_len { - columns.push(match mode { + let cur_col = match mode { AggregateMode::Partial => ScalarValue::iter_to_array( group_state_chunk.iter().map(|row_group_state| { - row_group_state.accumulator_set[x] + row_group_state.accumulator_set[idx] .state() - .map(|x| x[y].clone()) + .map(|elem| elem[y].clone()) .expect("unexpected accumulator state in hash aggregate") }), - ), + )?, AggregateMode::Final | AggregateMode::FinalPartitioned => { ScalarValue::iter_to_array(group_state_chunk.iter().map( |row_group_state| { - row_group_state.accumulator_set[x] + row_group_state.accumulator_set[idx] .evaluate() .expect("unexpected accumulator state in hash aggregate") }, - )) + ))? } - }?); - } - } - // cast output if needed (e.g. for types like Dictionary where - // the intermediate GroupByScalar type was not the same as the - // output - - let mut start_idx = 0; - for (_, begin, end) in indices[0].iter() { - for field in &output_fields[*begin..*end] { - columns[start_idx] = cast(&columns[start_idx], field.data_type())?; - start_idx += 1; + }; + // cast output if needed (e.g. for types like Dictionary where + // the intermediate GroupByScalar type was not the same as the + // output + let casted_col = cast( + &cur_col, + remaining_fields + .pop_front() + .ok_or_else(|| { + DataFusionError::Execution("expects to have field".to_string()) + })? + .data_type(), + )?; + columns.push(casted_col); } } From 32585fad2ddf7dd784188746d667898a065d665d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Jan 2023 14:54:10 +0300 Subject: [PATCH 12/19] readd arrow row converter --- .../src/physical_plan/aggregates/row_hash.rs | 92 +++++++++---------- datafusion/core/tests/sql/aggregates.rs | 29 ++++++ datafusion/core/tests/sql/window.rs | 52 ----------- 3 files changed, 70 insertions(+), 103 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index b387e2beb6ca2..03eb573015f93 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -24,6 +24,8 @@ use std::task::{Context, Poll}; use std::vec; use ahash::RandomState; +use arrow::row::{OwnedRow, RowConverter, SortField}; +use datafusion_physical_expr::hash_utils::create_hashes; use futures::stream::BoxStream; use futures::stream::{Stream, StreamExt}; @@ -39,21 +41,21 @@ use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use arrow::array::new_null_array; +use arrow::compute::cast; +use arrow::datatypes::{DataType, Schema}; +use arrow::{array::ArrayRef, compute}; use arrow::{ - array::{new_null_array, Array, ArrayRef, UInt32Builder}, - compute, - compute::cast, - datatypes::{DataType, Schema, SchemaRef}, + array::{Array, UInt32Builder}, error::{ArrowError, Result as ArrowResult}, - record_batch::RecordBatch, }; +use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use datafusion_common::{DataFusionError, ScalarValue}; use datafusion_expr::Accumulator; -use datafusion_physical_expr::hash_utils::create_hashes; use datafusion_row::accessor::RowAccessor; use datafusion_row::layout::RowLayout; use datafusion_row::reader::{read_row, RowReader}; -use datafusion_row::{row_supported, MutableRecordBatch, RowType}; +use datafusion_row::{MutableRecordBatch, RowType}; use hashbrown::raw::RawTable; /// Grouping aggregate with row-format aggregation states inside. @@ -94,6 +96,7 @@ struct GroupedHashAggregateStreamInner { group_by: PhysicalGroupBy, row_accumulators: Vec, + row_converter: RowConverter, row_aggr_schema: SchemaRef, row_aggr_layout: Arc, @@ -115,10 +118,6 @@ fn aggr_state_schema(aggr_expr: &[Arc]) -> Result Ok(Arc::new(Schema::new(fields))) } -fn is_supported(expr: &Arc, group_schema: &Schema) -> bool { - expr.row_accumulator_supported() && row_supported(group_schema, RowType::Compact) -} - impl GroupedHashAggregateStream { /// Create a new GroupedHashAggregateStream #[allow(clippy::too_many_arguments)] @@ -135,7 +134,6 @@ impl GroupedHashAggregateStream { ) -> Result { let timer = baseline_metrics.elapsed_compute().timer(); - let group_schema = group_schema(&schema, group_by.expr.len()); let mut start_idx = group_by.expr.len(); let mut row_aggr_expr = vec![]; let mut row_agg_indices = vec![]; @@ -157,7 +155,7 @@ impl GroupedHashAggregateStream { AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; - if is_supported(expr, &group_schema) { + if expr.row_accumulator_supported() { row_aggregate_expressions.push(others); row_agg_indices.push((idx, start_idx, start_idx + n_fields)); row_aggr_expr.push(expr.clone()); @@ -173,6 +171,15 @@ impl GroupedHashAggregateStream { let row_aggr_schema = aggr_state_schema(&row_aggr_expr)?; + let group_schema = group_schema(&schema, group_by.expr.len()); + let row_converter = RowConverter::new( + group_schema + .fields() + .iter() + .map(|f| SortField::new(f.data_type().clone())) + .collect(), + )?; + let row_aggr_layout = Arc::new(RowLayout::new(&row_aggr_schema, RowType::WordAligned)); @@ -192,6 +199,7 @@ impl GroupedHashAggregateStream { group_by, normal_aggr_expr, row_accumulators, + row_converter, row_aggr_schema, row_aggr_layout, baseline_metrics, @@ -218,6 +226,7 @@ impl GroupedHashAggregateStream { &this.group_by, &this.normal_aggr_expr, &mut this.row_accumulators, + &mut this.row_converter, this.row_aggr_layout.clone(), batch, &mut this.row_aggr_state, @@ -242,6 +251,7 @@ impl GroupedHashAggregateStream { let timer = this.baseline_metrics.elapsed_compute().timer(); let result = create_batch_from_map( &this.mode, + &this.row_converter, &this.row_aggr_schema, this.batch_size, this.row_group_skip_position, @@ -249,7 +259,6 @@ impl GroupedHashAggregateStream { &mut this.row_accumulators, &this.schema, &this.indices, - this.group_by.expr.len(), ); timer.done(); @@ -309,6 +318,7 @@ fn group_aggregate_batch( grouping_set: &PhysicalGroupBy, normal_aggr_expr: &[Arc], row_accumulators: &mut [RowAccumulatorItem], + row_converter: &mut RowConverter, state_layout: Arc, batch: RecordBatch, aggr_state: &mut RowAggregationState, @@ -332,7 +342,10 @@ fn group_aggregate_batch( let row_aggr_input_values = evaluate_many(row_aggregate_expressions, &batch)?; let normal_aggr_input_values = evaluate_many(normal_aggregate_expressions, &batch)?; + let row_converter_size_pre = row_converter.size(); for group_values in &group_by_values { + let group_rows = row_converter.convert_columns(group_values)?; + // 1.1 construct the key from the group values // 1.2 construct the mapping key if it does not exist // 1.3 add the row' index to `indices` @@ -350,11 +363,7 @@ fn group_aggregate_batch( // actually the same key value as the group in // existing_idx (aka group_values @ row) let group_state = &row_group_states[*group_idx]; - let group_by_values = get_at_row(group_values, row).unwrap(); - group_by_values - .iter() - .zip(group_state.group_by_values.iter()) - .all(|(lhs, rhs)| lhs.eq(rhs)) + group_rows.row(row) == group_state.group_by_values.row() }); match entry { @@ -373,12 +382,11 @@ fn group_aggregate_batch( } // 1.2 Need to create new entry None => { - let group_by_values = get_at_row(group_values, row).unwrap(); let accumulator_set = aggregates::create_accumulators(normal_aggr_expr)?; // Add new entry to group_states and save newly created index let group_state = RowGroupState { - group_by_values: group_by_values.into_boxed_slice(), + group_by_values: group_rows.row(row).owned(), aggregation_buffer: vec![0; state_layout.fixed_part_width()], accumulator_set, indices: vec![row as u32], // 1.3 @@ -387,11 +395,8 @@ fn group_aggregate_batch( // NOTE: do NOT include the `RowGroupState` struct size in here because this is captured by // `group_states` (see allocation down below) - row_allocated += group_state - .group_by_values - .iter() - .map(|sv| sv.size()) - .sum::() + row_allocated += (std::mem::size_of::() + * group_state.group_by_values.as_ref().len()) + (std::mem::size_of::() * group_state.aggregation_buffer.capacity()) + (std::mem::size_of::() * group_state.indices.capacity()); @@ -549,7 +554,7 @@ fn group_aggregate_batch( Ok::<(), DataFusionError>(()) })?; } - + row_allocated += row_converter.size().saturating_sub(row_converter_size_pre); Ok(row_allocated + normal_allocated) } @@ -557,7 +562,7 @@ fn group_aggregate_batch( #[derive(Debug)] pub struct RowGroupState { /// The actual group by values, stored sequentially - pub group_by_values: Box<[ScalarValue]>, + group_by_values: OwnedRow, // Accumulator state, stored sequentially pub aggregation_buffer: Vec, @@ -602,6 +607,7 @@ impl std::fmt::Debug for RowAggregationState { #[allow(clippy::too_many_arguments)] fn create_batch_from_map( mode: &AggregateMode, + converter: &RowConverter, aggr_schema: &Schema, batch_size: usize, skip_items: usize, @@ -609,7 +615,6 @@ fn create_batch_from_map( row_accumulators: &mut [RowAccumulatorItem], output_schema: &Schema, indices: &[Vec<(usize, usize, usize)>], - num_group_expr: usize, ) -> ArrowResult> { if skip_items > row_aggr_state.group_states.len() { return Ok(None); @@ -730,15 +735,12 @@ fn create_batch_from_map( } } - let mut res = (0..num_group_expr) - .map(|idx| { - ScalarValue::iter_to_array( - group_state_chunk - .iter() - .map(|gs| gs.group_by_values[idx].clone()), - ) - }) - .collect::>>()?; + let group_buffers = group_state_chunk + .iter() + .map(|gs| gs.group_by_values.row()) + .collect::>(); + let mut res: Vec = converter.convert_rows(group_buffers)?; + let extra: usize = indices .iter() .flatten() @@ -763,11 +765,7 @@ fn create_batch_from_map( )?)) } -pub fn read_as_batch( - rows: &[Vec], - schema: &Schema, - row_type: RowType, -) -> Vec { +fn read_as_batch(rows: &[Vec], schema: &Schema, row_type: RowType) -> Vec { let row_num = rows.len(); let mut output = MutableRecordBatch::new(row_num, Arc::new(schema.clone())); let mut row = RowReader::new(schema, row_type); @@ -779,11 +777,3 @@ pub fn read_as_batch( output.output_as_columns() } - -fn get_at_row(grouping_set_values: &[ArrayRef], row: usize) -> Result> { - // Copy group values out of arrays into `ScalarValue`s - grouping_set_values - .iter() - .map(|col| ScalarValue::try_from_array(col, row)) - .collect::>>() -} diff --git a/datafusion/core/tests/sql/aggregates.rs b/datafusion/core/tests/sql/aggregates.rs index 9911df9c7ff85..47bbffaefcb95 100644 --- a/datafusion/core/tests/sql/aggregates.rs +++ b/datafusion/core/tests/sql/aggregates.rs @@ -1058,3 +1058,32 @@ async fn aggregate_with_alias() -> Result<()> { ); Ok(()) } + +#[tokio::test] +async fn test_accumulator_row_accumulator() -> Result<()> { + let config = SessionConfig::new(); + let ctx = SessionContext::with_config(config); + register_aggregate_csv(&ctx).await?; + + let sql = "SELECT c1, c2, MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 + FROM aggregate_test_100 + GROUP BY c1, c2 + ORDER BY c1, c2 + LIMIT 5"; + + let actual = execute_to_batches(&ctx, sql).await; + let expected = vec![ + "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", + "| c1 | c2 | min1 | min2 | max1 | max2 | avg1 | min3 | cnt1 | sum1 |", + "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", + "| a | 1 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 774637006 | waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs | 4015442341 | 2437927011 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 5 | 6094771121.5 |", + "| a | 2 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 145294611 | ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 | 3717551163 | 2267588664 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 3 | 3401364777 |", + "| a | 3 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 431948861 | oLZ21P2JEDooxV1pU31cIxQHEeeoLu | 3998790955 | 2225685115.1666665 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 6 | 6676994872.5 |", + "| a | 4 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 466439833 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 2502326480 | 1655431654 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 4 | 3310812222.5 |", + "| a | 5 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 141047417 | QJYm7YRA3YetcBHI5wkMZeLXVmfuNy | 2496054700 | 1216992989.6666667 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 3 | 1825431770 |", + "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", + ]; + assert_batches_eq!(expected, &actual); + + Ok(()) +} diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index e88aa224a7127..1167d57a4ffb2 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2631,55 +2631,3 @@ mod tests { Ok(()) } } - -#[tokio::test] -async fn test_accumulator_row_accumulator() -> Result<()> { - let config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); - register_aggregate_csv(&ctx).await?; - - let sql = "SELECT c1, c2, MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 - FROM aggregate_test_100 - GROUP BY c1, c2 - ORDER BY c1, c2 - LIMIT 5"; - - let actual = execute_to_batches(&ctx, sql).await; - let expected = vec![ - "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", - "| c1 | c2 | min1 | min2 | max1 | max2 | avg1 | min3 | cnt1 | sum1 |", - "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", - "| a | 1 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 774637006 | waIGbOGl1PM6gnzZ4uuZt4E2yDWRHs | 4015442341 | 2437927011 | 0keZ5G8BffGwgF2RwQD59TFzMStxCB | 5 | 6094771121.5 |", - "| a | 2 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 145294611 | ukyD7b0Efj7tNlFSRmzZ0IqkEzg2a8 | 3717551163 | 2267588664 | b3b9esRhTzFEawbs6XhpKnD9ojutHB | 3 | 3401364777 |", - "| a | 3 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 431948861 | oLZ21P2JEDooxV1pU31cIxQHEeeoLu | 3998790955 | 2225685115.1666665 | Amn2K87Db5Es3dFQO9cw9cvpAM6h35 | 6 | 6676994872.5 |", - "| a | 4 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 466439833 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 2502326480 | 1655431654 | KJFcmTVjdkCMv94wYCtfHMFhzyRsmH | 4 | 3310812222.5 |", - "| a | 5 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 141047417 | QJYm7YRA3YetcBHI5wkMZeLXVmfuNy | 2496054700 | 1216992989.6666667 | MeSTAXq8gVxVjbEjgkvU9YLte0X9uE | 3 | 1825431770 |", - "+----+----+--------------------------------+-----------+--------------------------------+------------+--------------------+--------------------------------+------+--------------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} - -#[tokio::test] -async fn test_accumulator_row_accumulator_v2() -> Result<()> { - let config = SessionConfig::new(); - let ctx = SessionContext::with_config(config); - register_aggregate_csv(&ctx).await?; - - let sql = "SELECT MIN(c13) as min1, MIN(c9) as min2, MAX(c13) as max1, MAX(c9) as max2, AVG(c9) as avg1, MIN(c13) as min3, COUNT(C9) as cnt1, 0.5*SUM(c9-c8) as sum1 - FROM aggregate_test_100 - LIMIT 5"; - - let actual = execute_to_batches(&ctx, sql).await; - let expected = vec![ - "+--------------------------------+----------+--------------------------------+------------+--------------+--------------------------------+------+----------------+", - "| min1 | min2 | max1 | max2 | avg1 | min3 | cnt1 | sum1 |", - "+--------------------------------+----------+--------------------------------+------------+--------------+--------------------------------+------+----------------+", - "| 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 28774375 | ydkwycaISlYSlEq3TlkS2m15I2pcp8 | 4268716378 | 2220897700.6 | 0VVIHzxWtNOFLtnhjHEKjXaJOSLJfm | 100 | 111043376209.5 |", - "+--------------------------------+----------+--------------------------------+------------+--------------+--------------------------------+------+----------------+", - ]; - assert_batches_eq!(expected, &actual); - - Ok(()) -} From b6487d52f4cca3d721f8dcc4b2f0a9cc544d070e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Jan 2023 15:44:05 +0300 Subject: [PATCH 13/19] minor changes --- .../src/physical_plan/aggregates/row_hash.rs | 145 +++++++----------- 1 file changed, 58 insertions(+), 87 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 03eb573015f93..00d25f288bd5a 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -19,6 +19,7 @@ use std::cmp::min; use std::collections::VecDeque; +use std::ops::Range; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; @@ -41,9 +42,9 @@ use crate::physical_plan::{aggregates, AggregateExpr, PhysicalExpr}; use crate::physical_plan::{RecordBatchStream, SendableRecordBatchStream}; use crate::execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use arrow::array::new_null_array; +use arrow::array::{new_null_array, PrimitiveArray}; use arrow::compute::cast; -use arrow::datatypes::{DataType, Schema}; +use arrow::datatypes::{DataType, Schema, UInt32Type}; use arrow::{array::ArrayRef, compute}; use arrow::{ array::{Array, UInt32Builder}, @@ -107,7 +108,10 @@ struct GroupedHashAggregateStreamInner { /// if the result is chunked into batches, /// last offset is preserved for continuation. row_group_skip_position: usize, - indices: [Vec<(usize, usize, usize)>; 2], + /// keeps range for each accumulator in the field + /// first element in the array corresponds to normal accumulators + /// second element in the array corresponds to row accumulators + indices: [Vec>; 2], } fn aggr_state_schema(aggr_expr: &[Arc]) -> Result { @@ -146,22 +150,23 @@ impl GroupedHashAggregateStream { // col_idx_base to group expression count. let all_aggregate_expressions = aggregates::aggregate_expressions(&aggr_expr, &mode, start_idx)?; - for (idx, (expr, others)) in aggr_expr - .iter() - .zip(all_aggregate_expressions.into_iter()) - .enumerate() + for (expr, others) in aggr_expr.iter().zip(all_aggregate_expressions.into_iter()) { let n_fields = match mode { AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; + let aggr_range = Range { + start: start_idx, + end: start_idx + n_fields, + }; if expr.row_accumulator_supported() { row_aggregate_expressions.push(others); - row_agg_indices.push((idx, start_idx, start_idx + n_fields)); + row_agg_indices.push(aggr_range); row_aggr_expr.push(expr.clone()); } else { normal_aggregate_expressions.push(others); - normal_agg_indices.push((idx, start_idx, start_idx + n_fields)); + normal_agg_indices.push(aggr_range); normal_aggr_expr.push(expr.clone()); } start_idx += n_fields; @@ -435,43 +440,9 @@ fn group_aggregate_batch( } let batch_indices = batch_indices.finish(); - // `Take` all values based on indices into Arrays - let row_values: Vec>> = row_aggr_input_values - .iter() - .map(|array| { - array - .iter() - .map(|array| { - compute::take( - array.as_ref(), - &batch_indices, - None, // None: no index check - ) - .unwrap() - }) - .collect() - // 2.3 - }) - .collect(); + let row_values = get_at_indices(&row_aggr_input_values, &batch_indices); + let normal_values = get_at_indices(&normal_aggr_input_values, &batch_indices); - // `Take` all values based on indices into Arrays - let normal_values: Vec>> = normal_aggr_input_values - .iter() - .map(|array| { - array - .iter() - .map(|array| { - compute::take( - array.as_ref(), - &batch_indices, - None, // None: no index check - ) - .unwrap() - }) - .collect() - // 2.3 - }) - .collect(); // 2.1 for each key in this batch // 2.2 for each aggregation // 2.3 `slice` from each of its arrays the keys' values @@ -614,7 +585,7 @@ fn create_batch_from_map( row_aggr_state: &mut RowAggregationState, row_accumulators: &mut [RowAccumulatorItem], output_schema: &Schema, - indices: &[Vec<(usize, usize, usize)>], + indices: &[Vec>], ) -> ArrowResult> { if skip_items > row_aggr_state.group_states.len() { return Ok(None); @@ -652,15 +623,15 @@ fn create_batch_from_map( } } // We fill fields corresponding to row accumulators e.g indices[1] - let mut remaining_fields = vec![]; - for (_, begin, end) in indices[1].iter() { - for field in &output_fields[*begin..*end] { - remaining_fields.push(field); + let mut row_acc_fields = vec![]; + for Range { start, end } in indices[1].iter() { + for field in &output_fields[*start..*end] { + row_acc_fields.push(field); } } results .into_iter() - .zip(remaining_fields) + .zip(row_acc_fields) .map(|(scalars, field)| { if scalars.is_empty() { Ok(arrow::array::new_empty_array(field.data_type())) @@ -677,29 +648,15 @@ fn create_batch_from_map( } }; - // Calculate number/shape of state arrays - let accs = &row_aggr_state.group_states[0].accumulator_set; - let acc_state_lengths = match mode { - AggregateMode::Partial => accs - .iter() - .map(|a| a.state().map(|s| s.len())) - .collect::>>()?, - AggregateMode::Final | AggregateMode::FinalPartitioned => { - vec![1; accs.len()] - } - }; - // We fill fields corresponding to accumulators e.g indices[0] - let mut remaining_fields = VecDeque::new(); - for (_, begin, end) in indices[0].iter() { - for field in &output_fields[*begin..*end] { - remaining_fields.push_back(field); - } + let mut acc_fields = vec![]; + for Range { start, end } in indices[0].iter() { + acc_fields.push(&output_fields[*start..*end]); } // next, output aggregates: either intermediate state or final output let mut columns = vec![]; - for (idx, &state_len) in acc_state_lengths.iter().enumerate() { - for y in 0..state_len { + for (idx, acc_field) in acc_fields.iter().enumerate() { + for y in 0..acc_field.len() { let cur_col = match mode { AggregateMode::Partial => ScalarValue::iter_to_array( group_state_chunk.iter().map(|row_group_state| { @@ -722,15 +679,7 @@ fn create_batch_from_map( // cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output - let casted_col = cast( - &cur_col, - remaining_fields - .pop_front() - .ok_or_else(|| { - DataFusionError::Execution("expects to have field".to_string()) - })? - .data_type(), - )?; + let casted_col = cast(&cur_col, acc_field[y].data_type())?; columns.push(casted_col); } } @@ -744,18 +693,16 @@ fn create_batch_from_map( let extra: usize = indices .iter() .flatten() - .map(|(_, begin, end)| end - begin) + .map(|Range { start, end }| end - start) .sum(); let empty_arr = new_null_array(&DataType::Null, 1); res.extend(std::iter::repeat(empty_arr).take(extra)); - let results = vec![columns, row_columns]; - for (outer, cur_res) in results.into_iter().enumerate() { - let mut start_idx = 0; - for (_, begin, end) in indices[outer].iter() { - for elem in res.iter_mut().take(*end).skip(*begin) { - *elem = cur_res[start_idx].clone(); - start_idx += 1; + let results: Vec> = vec![columns.into(), row_columns.into()]; + for (outer, mut cur_res) in results.into_iter().enumerate() { + for Range { start, end } in indices[outer].iter() { + for elem in res.iter_mut().take(*end).skip(*start) { + *elem = cur_res.pop_front().expect("columns cannot be empty"); } } } @@ -777,3 +724,27 @@ fn read_as_batch(rows: &[Vec], schema: &Schema, row_type: RowType) -> Vec>], + batch_indices: &PrimitiveArray, +) -> Vec>> { + let res: Vec>> = input_values + .iter() + .map(|array| { + array + .iter() + .map(|array| { + compute::take( + array.as_ref(), + batch_indices, + None, // None: no index check + ) + .unwrap() + }) + .collect() + // 2.3 + }) + .collect(); + res +} From fdad3ef9b713b8e1decd9cf01d9bc85ec15b8331 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Jan 2023 17:34:27 +0300 Subject: [PATCH 14/19] simplify code --- .../src/physical_plan/aggregates/row_hash.rs | 21 ++++++------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 00d25f288bd5a..8541f43eb13c9 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -622,17 +622,12 @@ fn create_batch_from_map( results[i].push(acc.evaluate(&state_accessor)?); } } - // We fill fields corresponding to row accumulators e.g indices[1] - let mut row_acc_fields = vec![]; - for Range { start, end } in indices[1].iter() { - for field in &output_fields[*start..*end] { - row_acc_fields.push(field); - } - } results .into_iter() - .zip(row_acc_fields) - .map(|(scalars, field)| { + .enumerate() + .map(|(idx, scalars)| { + // Get corresponding field for row accumulator + let field = &output_fields[indices[1][idx].start]; if scalars.is_empty() { Ok(arrow::array::new_empty_array(field.data_type())) } else { @@ -648,14 +643,10 @@ fn create_batch_from_map( } }; - // We fill fields corresponding to accumulators e.g indices[0] - let mut acc_fields = vec![]; - for Range { start, end } in indices[0].iter() { - acc_fields.push(&output_fields[*start..*end]); - } // next, output aggregates: either intermediate state or final output let mut columns = vec![]; - for (idx, acc_field) in acc_fields.iter().enumerate() { + for (idx, Range { start, end }) in indices[0].iter().enumerate() { + let acc_field = &output_fields[*start..*end]; for y in 0..acc_field.len() { let cur_col = match mode { AggregateMode::Partial => ScalarValue::iter_to_array( From 2d46671be900cccb50827dc492caf2e49136e398 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Jan 2023 17:54:26 +0300 Subject: [PATCH 15/19] change loop order --- .../src/physical_plan/aggregates/row_hash.rs | 43 +++++++++---------- 1 file changed, 20 insertions(+), 23 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 8541f43eb13c9..76541930ddb3b 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -603,7 +603,6 @@ fn create_batch_from_map( return Ok(Some(RecordBatch::new_empty(schema))); } - let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); let mut state_buffers = group_state_chunk .iter() .map(|gs| gs.aggregation_buffer.clone()) @@ -615,31 +614,29 @@ fn create_batch_from_map( read_as_batch(&state_buffers, aggr_schema, RowType::WordAligned) } AggregateMode::Final | AggregateMode::FinalPartitioned => { - let mut results: Vec> = vec![vec![]; row_accumulators.len()]; - for buffer in state_buffers.iter_mut() { - state_accessor.point_to(0, buffer); - for (i, acc) in row_accumulators.iter().enumerate() { - results[i].push(acc.evaluate(&state_accessor)?); + let mut results = vec![]; + for (idx, acc) in row_accumulators.iter().enumerate() { + let mut state_accessor = + RowAccessor::new(aggr_schema, RowType::WordAligned); + let mut cur_col = vec![]; + for buffer in state_buffers.iter_mut() { + state_accessor.point_to(0, buffer); + cur_col.push(acc.evaluate(&state_accessor)?); } + // Get corresponding field for row accumulator + let field = &output_fields[indices[1][idx].start]; + let casted = if cur_col.is_empty() { + Ok(arrow::array::new_empty_array(field.data_type())) + } else { + let elem = ScalarValue::iter_to_array(cur_col)?; + // cast output if needed (e.g. for types like Dictionary where + // the intermediate GroupByScalar type was not the same as the + // output + cast(&elem, field.data_type()).map_err(DataFusionError::ArrowError) + }?; + results.push(casted); } results - .into_iter() - .enumerate() - .map(|(idx, scalars)| { - // Get corresponding field for row accumulator - let field = &output_fields[indices[1][idx].start]; - if scalars.is_empty() { - Ok(arrow::array::new_empty_array(field.data_type())) - } else { - let elem = ScalarValue::iter_to_array(scalars)?; - // cast output if needed (e.g. for types like Dictionary where - // the intermediate GroupByScalar type was not the same as the - // output - cast(&elem, field.data_type()) - .map_err(DataFusionError::ArrowError) - } - }) - .collect::>>()? } }; From de1940cfab416b2f3090eb4fdc3387ce3604e6b1 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 13 Jan 2023 15:17:08 -0600 Subject: [PATCH 16/19] Refactor Part 2 --- .../src/physical_plan/aggregates/row_hash.rs | 78 +++++++++---------- .../physical-expr/src/aggregate/count.rs | 7 +- 2 files changed, 41 insertions(+), 44 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 76541930ddb3b..132797d805a20 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -18,7 +18,6 @@ //! Hash aggregation through row format use std::cmp::min; -use std::collections::VecDeque; use std::ops::Range; use std::sync::Arc; use std::task::{Context, Poll}; @@ -146,8 +145,8 @@ impl GroupedHashAggregateStream { let mut normal_agg_indices = vec![]; let mut normal_aggregate_expressions = vec![]; // The expressions to evaluate the batch, one vec of expressions per aggregation. - // Assume create_schema() always put group columns in front of aggr columns, we set - // col_idx_base to group expression count. + // Assuming create_schema() always puts group columns in front of aggregation columns, we set + // col_idx_base to the group expression count. let all_aggregate_expressions = aggregates::aggregate_expressions(&aggr_expr, &mode, start_idx)?; for (expr, others) in aggr_expr.iter().zip(all_aggregate_expressions.into_iter()) @@ -272,16 +271,14 @@ impl GroupedHashAggregateStream { }; this.row_group_skip_position += this.batch_size; - match result { + return match result { Ok(Some(result)) => { - return Some(( - Ok(result.record_output(&this.baseline_metrics)), - this, - )); + let batch = result.record_output(&this.baseline_metrics); + Some((Ok(batch), this)) } - Ok(None) => return None, - Err(error) => return Some((Err(error), this)), - } + Ok(None) => None, + Err(error) => Some((Err(error), this)), + }; } }); @@ -618,23 +615,25 @@ fn create_batch_from_map( for (idx, acc) in row_accumulators.iter().enumerate() { let mut state_accessor = RowAccessor::new(aggr_schema, RowType::WordAligned); - let mut cur_col = vec![]; - for buffer in state_buffers.iter_mut() { - state_accessor.point_to(0, buffer); - cur_col.push(acc.evaluate(&state_accessor)?); - } + let current = state_buffers + .iter_mut() + .map(|buffer| { + state_accessor.point_to(0, buffer); + acc.evaluate(&state_accessor) + }) + .collect::>>()?; // Get corresponding field for row accumulator let field = &output_fields[indices[1][idx].start]; - let casted = if cur_col.is_empty() { + let result = if current.is_empty() { Ok(arrow::array::new_empty_array(field.data_type())) } else { - let elem = ScalarValue::iter_to_array(cur_col)?; + let item = ScalarValue::iter_to_array(current)?; // cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output - cast(&elem, field.data_type()).map_err(DataFusionError::ArrowError) + cast(&item, field.data_type()).map_err(DataFusionError::ArrowError) }?; - results.push(casted); + results.push(result); } results } @@ -642,33 +641,32 @@ fn create_batch_from_map( // next, output aggregates: either intermediate state or final output let mut columns = vec![]; - for (idx, Range { start, end }) in indices[0].iter().enumerate() { - let acc_field = &output_fields[*start..*end]; - for y in 0..acc_field.len() { - let cur_col = match mode { + for (idx, &Range { start, end }) in indices[0].iter().enumerate() { + for (field_idx, field) in output_fields[start..end].iter().enumerate() { + let current = match mode { AggregateMode::Partial => ScalarValue::iter_to_array( group_state_chunk.iter().map(|row_group_state| { row_group_state.accumulator_set[idx] .state() - .map(|elem| elem[y].clone()) - .expect("unexpected accumulator state in hash aggregate") + .map(|v| v[field_idx].clone()) + .expect("Unexpected accumulator state in hash aggregate") }), - )?, + ), AggregateMode::Final | AggregateMode::FinalPartitioned => { ScalarValue::iter_to_array(group_state_chunk.iter().map( |row_group_state| { row_group_state.accumulator_set[idx] .evaluate() - .expect("unexpected accumulator state in hash aggregate") + .expect("Unexpected accumulator state in hash aggregate") }, - ))? + )) } - }; + }?; // cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output - let casted_col = cast(&cur_col, acc_field[y].data_type())?; - columns.push(casted_col); + let result = cast(¤t, field.data_type())?; + columns.push(result); } } @@ -676,7 +674,7 @@ fn create_batch_from_map( .iter() .map(|gs| gs.group_by_values.row()) .collect::>(); - let mut res: Vec = converter.convert_rows(group_buffers)?; + let mut output: Vec = converter.convert_rows(group_buffers)?; let extra: usize = indices .iter() @@ -684,19 +682,19 @@ fn create_batch_from_map( .map(|Range { start, end }| end - start) .sum(); let empty_arr = new_null_array(&DataType::Null, 1); - res.extend(std::iter::repeat(empty_arr).take(extra)); + output.extend(std::iter::repeat(empty_arr).take(extra)); - let results: Vec> = vec![columns.into(), row_columns.into()]; - for (outer, mut cur_res) in results.into_iter().enumerate() { - for Range { start, end } in indices[outer].iter() { - for elem in res.iter_mut().take(*end).skip(*start) { - *elem = cur_res.pop_front().expect("columns cannot be empty"); + let results = [columns.into_iter(), row_columns.into_iter()]; + for (outer, mut current) in results.into_iter().enumerate() { + for &Range { start, end } in indices[outer].iter() { + for item in output.iter_mut().take(end).skip(start) { + *item = current.next().expect("Columns cannot be empty"); } } } Ok(Some(RecordBatch::try_new( Arc::new(output_schema.to_owned()), - res, + output, )?)) } diff --git a/datafusion/physical-expr/src/aggregate/count.rs b/datafusion/physical-expr/src/aggregate/count.rs index 5405ca7597e07..38b193ebf270d 100644 --- a/datafusion/physical-expr/src/aggregate/count.rs +++ b/datafusion/physical-expr/src/aggregate/count.rs @@ -202,10 +202,9 @@ impl RowAccumulator for CountRowAccumulator { } fn evaluate(&self, accessor: &RowAccessor) -> Result { - Ok(match accessor.get_u64_opt(self.state_index()) { - Some(cnt) => ScalarValue::Int64(Some(cnt as i64)), - None => ScalarValue::Int64(Some(0)), - }) + Ok(ScalarValue::Int64(Some( + accessor.get_u64_opt(self.state_index()).unwrap_or(0) as i64, + ))) } #[inline(always)] From 539e70d8ca6b8f5a26b0bfbd43619a71d5c83edf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 16 Jan 2023 18:45:07 +0300 Subject: [PATCH 17/19] Add comment --- .../src/physical_plan/aggregates/row_hash.rs | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index 132797d805a20..cb375fa2ca3ed 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -90,7 +90,9 @@ struct GroupedHashAggregateStreamInner { mode: AggregateMode, normal_aggr_expr: Vec>, row_aggr_state: RowAggregationState, + /// Aggregate Expressions where row_accumulation is not supported normal_aggregate_expressions: Vec>>, + /// Aggregate Expressions where row_accumulation is supported row_aggregate_expressions: Vec>>, group_by: PhysicalGroupBy, @@ -152,9 +154,11 @@ impl GroupedHashAggregateStream { for (expr, others) in aggr_expr.iter().zip(all_aggregate_expressions.into_iter()) { let n_fields = match mode { + // In partial aggregation we keep additional fields to be able to successfully merge in aggregation results in the downstream AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; + // Stores range of each expression let aggr_range = Range { start: start_idx, end: start_idx + n_fields, @@ -329,8 +333,10 @@ fn group_aggregate_batch( ) -> Result { // evaluate the grouping expressions let group_by_values = evaluate_group_by(grouping_set, &batch)?; - let mut row_allocated = 0usize; // track memory allocations + // memory allocated by row accumulators + let mut row_allocated = 0usize; + // memory allocated by normal accumulators let mut normal_allocated = 0usize; let RowAggregationState { map: row_map, @@ -403,6 +409,7 @@ fn group_aggregate_batch( * group_state.aggregation_buffer.capacity()) + (std::mem::size_of::() * group_state.indices.capacity()); + // Allocation done by normal accumulators normal_allocated += (std::mem::size_of::>() * group_state.accumulator_set.capacity()) + group_state @@ -535,7 +542,7 @@ pub struct RowGroupState { // Accumulator state, stored sequentially pub aggregation_buffer: Vec, - // Accumulator state, one for each aggregate + // Accumulator state, one for each aggregate that doesn't support row accumulation pub accumulator_set: Vec, /// scratch space used to collect indices for input rows in a @@ -582,6 +589,7 @@ fn create_batch_from_map( row_aggr_state: &mut RowAggregationState, row_accumulators: &mut [RowAccumulatorItem], output_schema: &Schema, + // Stores the location of each accumulator in the output_schema indices: &[Vec>], ) -> ArrowResult> { if skip_items > row_aggr_state.group_states.len() { @@ -600,12 +608,14 @@ fn create_batch_from_map( return Ok(Some(RecordBatch::new_empty(schema))); } + // Buffers for each distinct group (where row accumulator uses its memory) let mut state_buffers = group_state_chunk .iter() .map(|gs| gs.aggregation_buffer.clone()) .collect::>(); let output_fields = output_schema.fields(); + // Stores the result of row accumulators let row_columns = match mode { AggregateMode::Partial => { read_as_batch(&state_buffers, aggr_schema, RowType::WordAligned) @@ -640,6 +650,7 @@ fn create_batch_from_map( }; // next, output aggregates: either intermediate state or final output + // Stores the result of normal accumulators let mut columns = vec![]; for (idx, &Range { start, end }) in indices[0].iter().enumerate() { for (field_idx, field) in output_fields[start..end].iter().enumerate() { @@ -670,12 +681,14 @@ fn create_batch_from_map( } } + // Stores the group by fields let group_buffers = group_state_chunk .iter() .map(|gs| gs.group_by_values.row()) .collect::>(); let mut output: Vec = converter.convert_rows(group_buffers)?; + // The size of the place occupied by row and normal accumulators let extra: usize = indices .iter() .flatten() @@ -684,6 +697,8 @@ fn create_batch_from_map( let empty_arr = new_null_array(&DataType::Null, 1); output.extend(std::iter::repeat(empty_arr).take(extra)); + // Write normal accumulators results and row accumulators results to the corresponding location in + // the output schema let results = [columns.into_iter(), row_columns.into_iter()]; for (outer, mut current) in results.into_iter().enumerate() { for &Range { start, end } in indices[outer].iter() { From fb53ed31393654f93620c2c95e1a87f0f8c752da Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 16 Jan 2023 13:06:31 -0600 Subject: [PATCH 18/19] Remove unnecessary variable, improve comments, try triggering CI --- .../src/physical_plan/aggregates/row_hash.rs | 40 +++++++++---------- 1 file changed, 19 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index cb375fa2ca3ed..dba6e1b7d50e8 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -90,9 +90,9 @@ struct GroupedHashAggregateStreamInner { mode: AggregateMode, normal_aggr_expr: Vec>, row_aggr_state: RowAggregationState, - /// Aggregate Expressions where row_accumulation is not supported + /// Aggregate expressions not supporting row accumulation normal_aggregate_expressions: Vec>>, - /// Aggregate Expressions where row_accumulation is supported + /// Aggregate expressions supporting row accumulation row_aggregate_expressions: Vec>>, group_by: PhysicalGroupBy, @@ -154,11 +154,12 @@ impl GroupedHashAggregateStream { for (expr, others) in aggr_expr.iter().zip(all_aggregate_expressions.into_iter()) { let n_fields = match mode { - // In partial aggregation we keep additional fields to be able to successfully merge in aggregation results in the downstream + // In partial aggregation, we keep additional fields in order to successfully + // merge aggregation results downstream. AggregateMode::Partial => expr.state_fields()?.len(), _ => 1, }; - // Stores range of each expression + // Stores range of each expression: let aggr_range = Range { start: start_idx, end: start_idx + n_fields, @@ -331,12 +332,11 @@ fn group_aggregate_batch( normal_aggregate_expressions: &[Vec>], row_aggregate_expressions: &[Vec>], ) -> Result { - // evaluate the grouping expressions + // Evaluate the grouping expressions: let group_by_values = evaluate_group_by(grouping_set, &batch)?; - // track memory allocations - // memory allocated by row accumulators + // Memory allocated by row accumulators: let mut row_allocated = 0usize; - // memory allocated by normal accumulators + // Memory allocated by normal accumulators: let mut normal_allocated = 0usize; let RowAggregationState { map: row_map, @@ -344,7 +344,7 @@ fn group_aggregate_batch( .. } = aggr_state; - // evaluate the aggregation expressions. + // Evaluate the aggregation expressions. // We could evaluate them after the `take`, but since we need to evaluate all // of them anyways, it is more performant to do it while they are together. let row_aggr_input_values = evaluate_many(row_aggregate_expressions, &batch)?; @@ -436,8 +436,8 @@ fn group_aggregate_batch( let mut batch_indices: UInt32Builder = UInt32Builder::with_capacity(0); let mut offsets = vec![0]; let mut offset_so_far = 0; - for group_idx in groups_with_rows.iter() { - let indices = &row_group_states[*group_idx].indices; + for &group_idx in groups_with_rows.iter() { + let indices = &row_group_states[group_idx].indices; batch_indices.append_slice(indices); offset_so_far += indices.len(); offsets.push(offset_so_far); @@ -608,14 +608,14 @@ fn create_batch_from_map( return Ok(Some(RecordBatch::new_empty(schema))); } - // Buffers for each distinct group (where row accumulator uses its memory) + // Buffers for each distinct group (i.e. row accumulator memories) let mut state_buffers = group_state_chunk .iter() .map(|gs| gs.aggregation_buffer.clone()) .collect::>(); let output_fields = output_schema.fields(); - // Stores the result of row accumulators + // Store row accumulator results (either final output or intermediate state): let row_columns = match mode { AggregateMode::Partial => { read_as_batch(&state_buffers, aggr_schema, RowType::WordAligned) @@ -649,8 +649,7 @@ fn create_batch_from_map( } }; - // next, output aggregates: either intermediate state or final output - // Stores the result of normal accumulators + // Store normal accumulator results (either final output or intermediate state): let mut columns = vec![]; for (idx, &Range { start, end }) in indices[0].iter().enumerate() { for (field_idx, field) in output_fields[start..end].iter().enumerate() { @@ -673,7 +672,7 @@ fn create_batch_from_map( )) } }?; - // cast output if needed (e.g. for types like Dictionary where + // Cast output if needed (e.g. for types like Dictionary where // the intermediate GroupByScalar type was not the same as the // output let result = cast(¤t, field.data_type())?; @@ -697,8 +696,8 @@ fn create_batch_from_map( let empty_arr = new_null_array(&DataType::Null, 1); output.extend(std::iter::repeat(empty_arr).take(extra)); - // Write normal accumulators results and row accumulators results to the corresponding location in - // the output schema + // Write results of both accumulator types to the corresponding location in + // the output schema: let results = [columns.into_iter(), row_columns.into_iter()]; for (outer, mut current) in results.into_iter().enumerate() { for &Range { start, end } in indices[outer].iter() { @@ -730,7 +729,7 @@ fn get_at_indices( input_values: &[Vec>], batch_indices: &PrimitiveArray, ) -> Vec>> { - let res: Vec>> = input_values + input_values .iter() .map(|array| { array @@ -746,6 +745,5 @@ fn get_at_indices( .collect() // 2.3 }) - .collect(); - res + .collect() } From 13f008d36d913587be813ccf18b702e7b4ca4a28 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 17 Jan 2023 14:20:25 +0300 Subject: [PATCH 19/19] address reviews --- .../src/physical_plan/aggregates/row_hash.rs | 23 ++++++++----------- 1 file changed, 10 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_plan/aggregates/row_hash.rs b/datafusion/core/src/physical_plan/aggregates/row_hash.rs index dba6e1b7d50e8..13d3a61dac75a 100644 --- a/datafusion/core/src/physical_plan/aggregates/row_hash.rs +++ b/datafusion/core/src/physical_plan/aggregates/row_hash.rs @@ -334,10 +334,8 @@ fn group_aggregate_batch( ) -> Result { // Evaluate the grouping expressions: let group_by_values = evaluate_group_by(grouping_set, &batch)?; - // Memory allocated by row accumulators: - let mut row_allocated = 0usize; - // Memory allocated by normal accumulators: - let mut normal_allocated = 0usize; + // Keep track of memory allocated: + let mut allocated = 0usize; let RowAggregationState { map: row_map, group_states: row_group_states, @@ -386,7 +384,7 @@ fn group_aggregate_batch( group_state .indices - .push_accounted(row as u32, &mut row_allocated); // remember this row + .push_accounted(row as u32, &mut allocated); // remember this row } // 1.2 Need to create new entry None => { @@ -403,14 +401,14 @@ fn group_aggregate_batch( // NOTE: do NOT include the `RowGroupState` struct size in here because this is captured by // `group_states` (see allocation down below) - row_allocated += (std::mem::size_of::() + allocated += (std::mem::size_of::() * group_state.group_by_values.as_ref().len()) + (std::mem::size_of::() * group_state.aggregation_buffer.capacity()) + (std::mem::size_of::() * group_state.indices.capacity()); // Allocation done by normal accumulators - normal_allocated += (std::mem::size_of::>() + allocated += (std::mem::size_of::>() * group_state.accumulator_set.capacity()) + group_state .accumulator_set @@ -422,10 +420,10 @@ fn group_aggregate_batch( row_map.insert_accounted( (hash, group_idx), |(hash, _group_index)| *hash, - &mut row_allocated, + &mut allocated, ); - row_group_states.push_accounted(group_state, &mut row_allocated); + row_group_states.push_accounted(group_state, &mut allocated); groups_with_rows.push(group_idx); } @@ -517,7 +515,7 @@ fn group_aggregate_batch( } }; let size_post = accumulator.size(); - normal_allocated += size_post.saturating_sub(size_pre); + allocated += size_post.saturating_sub(size_pre); res }) // 2.5 @@ -529,8 +527,8 @@ fn group_aggregate_batch( Ok::<(), DataFusionError>(()) })?; } - row_allocated += row_converter.size().saturating_sub(row_converter_size_pre); - Ok(row_allocated + normal_allocated) + allocated += row_converter.size().saturating_sub(row_converter_size_pre); + Ok(allocated) } /// The state that is built for each output group. @@ -743,7 +741,6 @@ fn get_at_indices( .unwrap() }) .collect() - // 2.3 }) .collect() }