diff --git a/datafusion-examples/examples/udf/advanced_udaf.rs b/datafusion-examples/examples/udf/advanced_udaf.rs index f1651dbf28913..067ec1e0ec389 100644 --- a/datafusion-examples/examples/udf/advanced_udaf.rs +++ b/datafusion-examples/examples/udf/advanced_udaf.rs @@ -18,7 +18,7 @@ //! See `main.rs` for how to run it. use arrow::datatypes::{Field, Schema}; -use datafusion::physical_expr::NullState; +use datafusion::physical_expr::FlatNullState; use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; use std::sync::Arc; @@ -215,7 +215,7 @@ struct GeometricMeanGroupsAccumulator { prods: Vec, /// Track nulls in the input / filters - null_state: NullState, + null_state: FlatNullState, } impl GeometricMeanGroupsAccumulator { @@ -225,7 +225,7 @@ impl GeometricMeanGroupsAccumulator { return_data_type: DataType::Float64, counts: vec![], prods: vec![], - null_state: NullState::new(), + null_state: FlatNullState::new(None), } } } @@ -246,13 +246,17 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { // increment counts, update sums self.counts.resize(total_num_groups, 0); self.prods.resize(total_num_groups, 1.0); - // Use the `NullState` structure to generate specialized code for null / non null input elements + // Use the `NullState` structure to generate specialized code for null / non null input elements. + // `block_id` is ignored in `value_fn`, because `AvgGroupsAccumulator` + // still not support blocked groups. + // More details can see `GroupsAccumulator::supports_blocked_groups`. self.null_state.accumulate( group_indices, values, opt_filter, total_num_groups, - |group_index, new_value| { + |_, group_index, new_value| { + let group_index = group_index as usize; let prod = &mut self.prods[group_index]; *prod = prod.mul_wrapping(new_value); @@ -277,13 +281,16 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { let partial_counts = values[1].as_primitive::(); // update counts with partial counts self.counts.resize(total_num_groups, 0); + // `block_id` is ignored in `value_fn`, because `AvgGroupsAccumulator` + // still not support blocked groups. + // More details can see `GroupsAccumulator::supports_blocked_groups`. self.null_state.accumulate( group_indices, partial_counts, opt_filter, total_num_groups, - |group_index, partial_count| { - self.counts[group_index] += partial_count; + |_, group_index, partial_count| { + self.counts[group_index as usize] += partial_count; }, ); @@ -294,8 +301,8 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { partial_prods, opt_filter, total_num_groups, - |group_index, new_value: ::Native| { - let prod = &mut self.prods[group_index]; + |_, group_index, new_value: ::Native| { + let prod = &mut self.prods[group_index as usize]; *prod = prod.mul_wrapping(new_value); }, ); diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 0eec3f948034a..b51b321fb5306 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -660,6 +660,16 @@ config_namespace! { /// the remote end point. pub objectstore_writer_buffer_size: usize, default = 10 * 1024 * 1024 + /// Should DataFusion use a blocked approach to manage grouping state. + /// By default, the blocked approach is used which + /// allocates capacity based on a predefined block size firstly. + /// When the block reaches its limit, we allocate a new block (also with + /// the same predefined block size based capacity) instead of expanding + /// the current one and copying the data. + /// If `false`, a single allocation approach is used, where + /// values are managed within a single large memory block. + /// As this block grows, it often triggers numerous copies, resulting in poor performance. + pub enable_aggregation_blocked_groups: bool, default = true /// Whether to enable ANSI SQL mode. /// /// The flag is experimental and relevant only for DataFusion Spark built-in functions diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 4726e7c4aca5c..f895c2bda0641 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -18,10 +18,12 @@ use std::sync::Arc; use super::record_batch_generator::get_supported_types_columns; -use crate::fuzz_cases::aggregation_fuzzer::query_builder::QueryBuilder; use crate::fuzz_cases::aggregation_fuzzer::{ AggregationFuzzerBuilder, DatasetGeneratorConfig, }; +use crate::fuzz_cases::aggregation_fuzzer::{ + SessionContextOptions, query_builder::QueryBuilder, +}; use arrow::array::{ Array, ArrayRef, AsArray, Int32Array, Int64Array, RecordBatch, StringArray, @@ -230,6 +232,60 @@ async fn test_median() { .await; } +// Testing `blocked groups optimization` +// Details of this optimization can see: +// https://github.com/apache/datafusion/issues/7065 +// +// To ensure the blocked groups path is actually exercised, we must satisfy +// *all* conditions checked by `can_enable_blocked_groups`: +// +// 1. `GroupOrdering::None` — disable sort hints (`sort_hint: Some(false)`) +// so the aggregation is not streaming. +// 2. `OutOfMemoryMode::ReportError` — no memory limit / spilling (default). +// 3. `group_values.supports_blocked_groups() == true` +// — only `GroupValuesPrimitive` returns true, so we restrict to +// *single* numeric group-by column (`with_max_group_by_columns(1)` +// + `set_group_by_columns(numeric_columns())`). +// 4. Every accumulator `supports_blocked_groups() == true` +// — currently only sum/min/max support this, so we limit aggregates +// to those functions with numeric arguments. +// 5. Config knob enabled — force it via `enable_blocked_groups: Some(true)`. +// +// With all five conditions met, `enable_blocked_groups` is guaranteed true +// and `alter_block_size` will be called on both group values and accumulators. +#[tokio::test(flavor = "multi_thread")] +async fn test_blocked_groups_optimization() { + let data_gen_config = baseline_config(); + + // Test `Numeric aggregation` + `Single group by` + let aggr_functions = ["sum", "min", "max"]; + let aggr_arguments = data_gen_config.numeric_columns(); + let groups_by_columns = data_gen_config.numeric_columns(); + + let mut query_builder = QueryBuilder::new() + .with_table_name("fuzz_table") + .with_aggregate_arguments(aggr_arguments) + .set_group_by_columns(groups_by_columns) + .with_min_group_by_columns(1) + .with_max_group_by_columns(1) + .with_no_grouping(false); + + for func in aggr_functions { + query_builder = query_builder.with_aggregate_function(func); + } + + AggregationFuzzerBuilder::from(data_gen_config) + .add_query_builder(query_builder) + .session_context_options(SessionContextOptions { + enable_blocked_groups: Some(true), + sort_hint: Some(false), + ..Default::default() + }) + .build() + .run() + .await; +} + /// Return a standard set of columns for testing data generation /// /// Includes numeric and string types diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs index fe31098622c58..0402d2404752b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs @@ -62,10 +62,33 @@ pub struct SessionContextGenerator { /// The upper bound of the randomly generated target partitions, /// and the lower bound will be 1 max_target_partitions: usize, + + /// Force-enable or force-disable specific options, overriding random generation + options: SessionContextOptions, +} + +/// Options to force-enable or force-disable specific session config knobs. +/// +/// - `None` → randomized by the fuzzer (default behavior) +/// - `Some(true)` → always enabled +/// - `Some(false)` → always disabled +#[derive(Debug, Clone, Default)] +pub struct SessionContextOptions { + pub skip_partial: Option, + pub sort_hint: Option, + pub enable_blocked_groups: Option, } impl SessionContextGenerator { pub fn new(dataset_ref: Arc, table_name: &str) -> Self { + Self::new_with_options(dataset_ref, table_name, SessionContextOptions::default()) + } + + pub fn new_with_options( + dataset_ref: Arc, + table_name: &str, + options: SessionContextOptions, + ) -> Self { let candidate_skip_partial_params = vec![ SkipPartialParams::ensure_trigger(), SkipPartialParams::ensure_not_trigger(), @@ -80,6 +103,7 @@ impl SessionContextGenerator { max_batch_size, candidate_skip_partial_params, max_target_partitions, + options, } } } @@ -102,6 +126,7 @@ impl SessionContextGenerator { target_partitions, skip_partial_params, sort_hint: false, + enable_aggregation_blocked_groups: false, table_name: self.table_name.clone(), table_provider: Arc::new(provider), }; @@ -126,13 +151,21 @@ impl SessionContextGenerator { let target_partitions = rng.random_range(1..=self.max_target_partitions); - let skip_partial_params_idx = - rng.random_range(0..self.candidate_skip_partial_params.len()); - let skip_partial_params = - self.candidate_skip_partial_params[skip_partial_params_idx]; + let skip_partial_params = match self.options.skip_partial { + Some(true) => SkipPartialParams::ensure_trigger(), + Some(false) => SkipPartialParams::ensure_not_trigger(), + None => { + let idx = rng.random_range(0..self.candidate_skip_partial_params.len()); + self.candidate_skip_partial_params[idx] + } + }; + let sort_hint_enabled = self + .options + .sort_hint + .unwrap_or_else(|| rng.random_bool(0.5)); let (provider, sort_hint) = - if rng.random_bool(0.5) && !self.dataset.sort_keys.is_empty() { + if sort_hint_enabled && !self.dataset.sort_keys.is_empty() { // Sort keys exist and random to push down let sort_exprs = self .dataset @@ -145,11 +178,17 @@ impl SessionContextGenerator { (provider, false) }; + let enable_aggregation_blocked_groups = self + .options + .enable_blocked_groups + .unwrap_or_else(|| rng.random_bool(0.5)); + let builder = GeneratedSessionContextBuilder { batch_size, target_partitions, sort_hint, skip_partial_params, + enable_aggregation_blocked_groups, table_name: self.table_name.clone(), table_provider: Arc::new(provider), }; @@ -173,6 +212,7 @@ struct GeneratedSessionContextBuilder { target_partitions: usize, sort_hint: bool, skip_partial_params: SkipPartialParams, + enable_aggregation_blocked_groups: bool, table_name: String, table_provider: Arc, } @@ -197,6 +237,10 @@ impl GeneratedSessionContextBuilder { "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", &ScalarValue::Float64(Some(self.skip_partial_params.ratio_threshold)), ); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(self.enable_aggregation_blocked_groups)), + ); let ctx = SessionContext::new_with_config(session_config); ctx.register_table(self.table_name, self.table_provider)?; @@ -206,6 +250,7 @@ impl GeneratedSessionContextBuilder { target_partitions: self.target_partitions, sort_hint: self.sort_hint, skip_partial_params: self.skip_partial_params, + enable_aggregation_blocked_groups: self.enable_aggregation_blocked_groups, }; Ok(SessionContextWithParams { ctx, params }) @@ -220,6 +265,7 @@ pub struct SessionContextParams { target_partitions: usize, sort_hint: bool, skip_partial_params: SkipPartialParams, + enable_aggregation_blocked_groups: bool, } /// Partial skipping parameters diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs index 430762b1c28db..e473355e40bd4 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs @@ -26,7 +26,9 @@ use rand::{Rng, rng}; use crate::fuzz_cases::aggregation_fuzzer::query_builder::QueryBuilder; use crate::fuzz_cases::aggregation_fuzzer::{ check_equality_of_batches, - context_generator::{SessionContextGenerator, SessionContextWithParams}, + context_generator::{ + SessionContextGenerator, SessionContextOptions, SessionContextWithParams, + }, data_generator::{Dataset, DatasetGenerator, DatasetGeneratorConfig}, run_sql, }; @@ -50,6 +52,9 @@ pub struct AggregationFuzzerBuilder { /// See `data_gen_rounds` in [`AggregationFuzzer`], default 16 data_gen_rounds: usize, + + /// Session context options to force-enable or force-disable specific knobs + session_context_options: SessionContextOptions, } impl AggregationFuzzerBuilder { @@ -59,6 +64,7 @@ impl AggregationFuzzerBuilder { table_name: None, data_gen_config: None, data_gen_rounds: 16, + session_context_options: SessionContextOptions::default(), } } @@ -90,6 +96,11 @@ impl AggregationFuzzerBuilder { self } + pub fn session_context_options(mut self, options: SessionContextOptions) -> Self { + self.session_context_options = options; + self + } + pub fn build(self) -> AggregationFuzzer { assert!(!self.candidate_sqls.is_empty()); let candidate_sqls = self.candidate_sqls; @@ -104,6 +115,7 @@ impl AggregationFuzzerBuilder { table_name, dataset_generator, data_gen_rounds, + session_context_options: self.session_context_options, } } } @@ -139,6 +151,9 @@ pub struct AggregationFuzzer { /// It is suggested to set value 2x or more bigger than num of /// `candidate_sqls` for better test coverage. data_gen_rounds: usize, + + /// Session context options to force-enable or force-disable specific knobs + session_context_options: SessionContextOptions, } /// Query group including the tested dataset and its sql query @@ -210,8 +225,11 @@ impl AggregationFuzzer { let mut tasks = Vec::with_capacity(query_groups.len() * CTX_GEN_ROUNDS); for QueryGroup { dataset, sql } in query_groups { let dataset_ref = Arc::new(dataset); - let ctx_generator = - SessionContextGenerator::new(dataset_ref.clone(), &self.table_name); + let ctx_generator = SessionContextGenerator::new_with_options( + dataset_ref.clone(), + &self.table_name, + self.session_context_options.clone(), + ); // Generate the baseline context, and get the baseline result firstly let baseline_ctx_with_params = ctx_generator diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs index e7ce557d2267d..757f836ae87aa 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/mod.rs @@ -46,6 +46,7 @@ mod fuzzer; pub mod query_builder; pub use crate::fuzz_cases::record_batch_generator::ColumnDescr; +pub use context_generator::SessionContextOptions; pub use data_generator::DatasetGeneratorConfig; pub use fuzzer::*; diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs index 7bb6177c31010..a9e143754143b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs @@ -182,13 +182,11 @@ impl QueryBuilder { /// Add max columns num in group by(default: 3), for example if it is set to 1, /// the generated sql will group by at most 1 column - #[expect(dead_code)] pub fn with_max_group_by_columns(mut self, max_group_by_columns: usize) -> Self { self.max_group_by_columns = max_group_by_columns; self } - #[expect(dead_code)] pub fn with_min_group_by_columns(mut self, min_group_by_columns: usize) -> Self { self.min_group_by_columns = min_group_by_columns; self @@ -202,7 +200,6 @@ impl QueryBuilder { } /// Add if also test the no grouping aggregation case(default: true) - #[expect(dead_code)] pub fn with_no_grouping(mut self, no_grouping: bool) -> Self { self.no_grouping = no_grouping; self @@ -245,7 +242,6 @@ impl QueryBuilder { fn generate_query(&self) -> String { let group_by = self.random_group_by(); - dbg!(&group_by); let mut query = String::from("SELECT "); query.push_str(&group_by.join(", ")); if !group_by.is_empty() { diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 9053f7a8eab9f..6f86ddf71eca8 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -18,12 +18,12 @@ //! Vectorized [`GroupsAccumulator`] use arrow::array::{ArrayRef, BooleanArray}; -use datafusion_common::{Result, not_impl_err}; +use datafusion_common::{DataFusionError, Result, not_impl_err}; /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy, PartialEq, Eq)] pub enum EmitTo { - /// Emit all groups + /// Emit all groups, will clear all existing group indexes All, /// Emit only the first `n` groups and shift all existing group /// indexes down by `n`. @@ -31,6 +31,10 @@ pub enum EmitTo { /// For example, if `n=10`, group_index `0, 1, ... 9` are emitted /// and group indexes `10, 11, 12, ...` become `0, 1, 2, ...`. First(usize), + /// Emit next block in the blocked managed groups + /// + /// Similar as `Emit::All`, will also clear all existing group indexes + NextBlock, } impl EmitTo { @@ -39,6 +43,9 @@ impl EmitTo { /// remaining values in `v`. /// /// This avoids copying if Self::All + /// + /// NOTICE: only support emit strategies: `Self::All` and `Self::First` + /// pub fn take_needed(&self, v: &mut Vec) -> Vec { match self { Self::All => { @@ -52,6 +59,7 @@ impl EmitTo { std::mem::swap(v, &mut t); t } + Self::NextBlock => unreachable!("don't support take block in take_needed"), } } } @@ -253,4 +261,49 @@ pub trait GroupsAccumulator: Send + std::any::Any { /// This function is called once per batch, so it should be `O(n)` to /// compute, not `O(num_groups)` fn size(&self) -> usize; + + /// Returns `true` if this accumulator supports blocked groups. + /// + /// Blocked groups(or called blocked management approach) is an optimization + /// to reduce the cost of managing aggregation intermediate states. + /// + /// Here is brief introduction for two states management approaches: + /// - Blocked approach, states are stored and managed in multiple `Vec`s, + /// we call it `Block`s. Organize like this is for avoiding to resize `Vec` + /// and allocate a new `Vec` instead to reduce cost and get better performance. + /// When locating data in `Block`s, we need to use `block_id` to locate the + /// needed `Block` at first, and use `block_offset` to locate the needed + /// data in `Block` after. + /// + /// - Single approach, all states are stored and managed in a single large `Block`. + /// So when locating data, `block_id` will always be 0, and we only need `block_offset` + /// to locate data in the single `Block`. + /// + /// More details can see: + /// + /// + fn supports_blocked_groups(&self) -> bool { + false + } + + /// Alter the block size in the accumulator + /// + /// If the target block size is `None`, it will use a single big + /// block(can think it a `Vec`) to manage the state. + /// + /// If the target block size` is `Some(blk_size)`, it will try to + /// set the block size to `blk_size`, and the try will only success + /// when the accumulator has supported blocked mode. + /// + /// NOTICE: After altering block size, all data in previous will be cleared. + /// + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { + if block_size.is_some() { + return Err(DataFusionError::NotImplemented( + "this accumulator doesn't support blocked mode yet".to_string(), + )); + } + + Ok(()) + } } diff --git a/datafusion/ffi/src/udaf/groups_accumulator.rs b/datafusion/ffi/src/udaf/groups_accumulator.rs index 0dc8edbfe5a85..eb7ce342c25ad 100644 --- a/datafusion/ffi/src/udaf/groups_accumulator.rs +++ b/datafusion/ffi/src/udaf/groups_accumulator.rs @@ -448,6 +448,7 @@ impl GroupsAccumulator for ForeignGroupsAccumulator { pub enum FFI_EmitTo { All, First(usize), + NextBlock, } impl From for FFI_EmitTo { @@ -455,6 +456,7 @@ impl From for FFI_EmitTo { match value { EmitTo::All => Self::All, EmitTo::First(v) => Self::First(v), + EmitTo::NextBlock => Self::NextBlock, } } } @@ -464,6 +466,7 @@ impl From for EmitTo { match value { FFI_EmitTo::All => Self::All, FFI_EmitTo::First(v) => Self::First(v), + FFI_EmitTo::NextBlock => Self::NextBlock, } } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index ad2a21bb4733c..b2820996d5eec 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -19,7 +19,9 @@ //! Adapter that makes [`GroupsAccumulator`] out of [`Accumulator`] pub mod accumulate; +pub mod blocks; pub mod bool_op; +pub mod group_index_operations; pub mod nulls; pub mod prim_op; diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 25f52df61136f..2500671b67703 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,12 +19,65 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator -use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; +use std::fmt::Debug; +use std::marker::PhantomData; + use arrow::buffer::NullBuffer; use arrow::datatypes::ArrowPrimitiveType; +use arrow::{ + array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}, + buffer::BooleanBuffer, +}; use datafusion_expr_common::groups_accumulator::EmitTo; +use crate::aggregate::groups_accumulator::blocks::{Block, Blocks}; +use crate::aggregate::groups_accumulator::group_index_operations::{ + BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, +}; + +/// Newtype wrapper around [`BooleanBufferBuilder`] that implements [`Default`] +/// (and thus [`Block`]) so it can be used in [`Blocks`]. +#[derive(Debug)] +pub struct BooleanBlock(BooleanBufferBuilder); + +impl BooleanBlock { + pub fn new(capacity: usize) -> Self { + Self(BooleanBufferBuilder::new(capacity)) + } +} + +impl Default for BooleanBlock { + fn default() -> Self { + Self(BooleanBufferBuilder::new(0)) + } +} + +impl std::ops::Deref for BooleanBlock { + type Target = BooleanBufferBuilder; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl std::ops::DerefMut for BooleanBlock { + fn deref_mut(&mut self) -> &mut Self::Target { + &mut self.0 + } +} + +impl Block for BooleanBlock { + type T = bool; + + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { + self.0.append_n(fill_len, default_value); + } + + fn len(&self) -> usize { + self.0.len() + } +} /// If the input has nulls, then the accumulator must potentially /// handle each input null value specially (e.g. for `SUM` to mark the /// corresponding sum as null) @@ -42,7 +95,7 @@ pub enum SeenValues { }, // Some groups have not yet seen a non-null value Some { - values: BooleanBufferBuilder, + builder: Blocks, }, } @@ -60,27 +113,33 @@ impl SeenValues { /// /// The builder is then ensured to have at least `total_num_groups` length, /// with any new entries initialized to false. - fn get_builder(&mut self, total_num_groups: usize) -> &mut BooleanBufferBuilder { - match self { + fn get_big_enough_builder( + &mut self, + total_num_groups: usize, + block_size: Option, + ) -> &mut Blocks { + // If `self` is `SeenValues::All`, transition it to `SeenValues::Some` with `num_values trues` firstly, + // then return mutable reference to the builder. + // If `self` is `SeenValues::Some`, just directly return mutable reference to the builder. + let new_block = + |block_size: Option| BooleanBlock::new(block_size.unwrap_or(0)); + + let builder = match self { SeenValues::All { num_values } => { - let mut builder = BooleanBufferBuilder::new(total_num_groups); - builder.append_n(*num_values, true); - if total_num_groups > *num_values { - builder.append_n(total_num_groups - *num_values, false); - } - *self = SeenValues::Some { values: builder }; + let mut builder = Blocks::new(block_size); + builder.resize(*num_values, new_block, true); + *self = SeenValues::Some { builder }; match self { - SeenValues::Some { values } => values, + SeenValues::Some { builder } => builder, _ => unreachable!(), } } - SeenValues::Some { values } => { - if values.len() < total_num_groups { - values.append_n(total_num_groups - values.len(), false); - } - values - } - } + SeenValues::Some { builder } => builder, + }; + + // Ensure the builder has `total_num_groups` length and return it + builder.resize(total_num_groups, new_block, false); + builder } } @@ -110,7 +169,7 @@ impl SeenValues { /// /// [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator #[derive(Debug)] -pub struct NullState { +pub struct NullState { /// Have we seen any non-filtered input values for `group_index`? /// /// If `seen_values` is `SeenValues::Some(buffer)` and buffer\[i\] is true, have seen at least one non null @@ -121,18 +180,21 @@ pub struct NullState { /// /// If `seen_values` is `SeenValues::All`, all groups have seen at least one non null value seen_values: SeenValues, -} -impl Default for NullState { - fn default() -> Self { - Self::new() - } + /// Size of one seen values block, can be None if only desire single block + block_size: Option, + + /// phantom data for required type `` + _phantom: PhantomData, } -impl NullState { - pub fn new() -> Self { +impl NullState { + /// Create a new `NullState` + pub fn new(block_size: Option) -> Self { Self { seen_values: SeenValues::All { num_values: 0 }, + block_size, + _phantom: PhantomData, } } @@ -140,7 +202,7 @@ impl NullState { pub fn size(&self) -> usize { match &self.seen_values { SeenValues::All { .. } => 0, - SeenValues::Some { values } => values.capacity() / 8, + SeenValues::Some { builder: values } => values.size(), } } @@ -169,22 +231,33 @@ impl NullState { mut value_fn: F, ) where T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, + F: FnMut(u32, u64, T::Native) + Send, { // skip null handling if no nulls in input or accumulator if let SeenValues::All { num_values } = &mut self.seen_values && opt_filter.is_none() && values.null_count() == 0 { - accumulate(group_indices, values, None, value_fn); + accumulate(group_indices, values, None, |packed_index, value| { + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + value_fn(block_id, block_offset, value); + }); *num_values = total_num_groups; return; } - let seen_values = self.seen_values.get_builder(total_num_groups); - accumulate(group_indices, values, opt_filter, |group_index, value| { - seen_values.set_bit(group_index, true); - value_fn(group_index, value); + // Get big enough `seen_values_builder`(start everything at "not seen" valid) + let seen_values_builder = self + .seen_values + .get_big_enough_builder(total_num_groups, self.block_size); + accumulate(group_indices, values, opt_filter, |packed_index, value| { + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + value_fn(block_id, block_offset, value); + seen_values_builder.set_bit(block_id, block_offset, true); }); } @@ -206,7 +279,7 @@ impl NullState { total_num_groups: usize, mut value_fn: F, ) where - F: FnMut(usize, bool) + Send, + F: FnMut(u32, u64, bool) + Send, { let data = values.values(); assert_eq!(data.len(), group_indices.len()); @@ -216,16 +289,23 @@ impl NullState { && opt_filter.is_none() && values.null_count() == 0 { - group_indices - .iter() - .zip(data.iter()) - .for_each(|(&group_index, new_value)| value_fn(group_index, new_value)); + group_indices.iter().zip(data.iter()).for_each( + |(&packed_index, new_value)| { + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + value_fn(block_id, block_offset, new_value); + }, + ); *num_values = total_num_groups; return; } - let seen_values = self.seen_values.get_builder(total_num_groups); + // Get big enough `seen_values_builder`(start everything at "not seen" valid) + let seen_values_builder = self + .seen_values + .get_big_enough_builder(total_num_groups, self.block_size); // These could be made more performant by iterating in chunks of 64 bits at a time match (values.null_count() > 0, opt_filter) { @@ -234,9 +314,12 @@ impl NullState { // if we have previously seen nulls, ensure the null // buffer is big enough (start everything at valid) group_indices.iter().zip(data.iter()).for_each( - |(&group_index, new_value)| { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value) + |(&packed_index, new_value)| { + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + seen_values_builder.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); }, ) } @@ -247,10 +330,13 @@ impl NullState { .iter() .zip(data.iter()) .zip(nulls.iter()) - .for_each(|((&group_index, new_value), is_valid)| { + .for_each(|((&packed_index, new_value), is_valid)| { if is_valid { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + seen_values_builder.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); } }) } @@ -262,10 +348,13 @@ impl NullState { .iter() .zip(data.iter()) .zip(filter.iter()) - .for_each(|((&group_index, new_value), filter_value)| { + .for_each(|((&packed_index, new_value), filter_value)| { if let Some(true) = filter_value { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value); + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + seen_values_builder.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); } }) } @@ -276,57 +365,305 @@ impl NullState { .iter() .zip(group_indices.iter()) .zip(values.iter()) - .for_each(|((filter_value, &group_index), new_value)| { + .for_each(|((filter_value, &packed_index), new_value)| { if let Some(true) = filter_value && let Some(new_value) = new_value { - seen_values.set_bit(group_index, true); - value_fn(group_index, new_value) + let packed_index = packed_index as u64; + let block_id = O::get_block_id(packed_index); + let block_offset = O::get_block_offset(packed_index); + seen_values_builder.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); } }) } } } - /// Creates the a [`NullBuffer`] representing which group_indices - /// should have null values (because they never saw any values) - /// for the `emit_to` rows. - /// - /// resets the internal state appropriately pub fn build(&mut self, emit_to: EmitTo) -> Option { - match emit_to { - EmitTo::All => { - let old_seen = std::mem::take(&mut self.seen_values); - match old_seen { - SeenValues::All { .. } => None, - SeenValues::Some { mut values } => { - Some(NullBuffer::new(values.finish())) - } + // If `self` is `SeenValues::All`, just modify `num_values`. + // If `self` is `SeenValues::Some`, perform the emit logic of `builder`. + match &mut self.seen_values { + SeenValues::All { num_values } => match emit_to { + EmitTo::All => { + assert!( + self.block_size.is_none(), + "emit_to::All should only be used with flat groups" + ); + None } - } - EmitTo::First(n) => match &mut self.seen_values { - SeenValues::All { num_values } => { + EmitTo::First(n) => { + assert!( + self.block_size.is_none(), + "emit_to::First should only be used with flat groups" + ); *num_values = num_values.saturating_sub(n); None } - SeenValues::Some { .. } => { - let mut old_values = match std::mem::take(&mut self.seen_values) { - SeenValues::Some { values } => values, - _ => unreachable!(), - }; - let nulls = old_values.finish(); - let first_n_null = nulls.slice(0, n); - let remainder = nulls.slice(n, nulls.len() - n); - let mut new_builder = BooleanBufferBuilder::new(remainder.len()); - new_builder.append_buffer(&remainder); - self.seen_values = SeenValues::Some { - values: new_builder, - }; - Some(NullBuffer::new(first_n_null)) + EmitTo::NextBlock => { + assert!( + self.block_size.is_some(), + "emit_to::NextBlock should only be used with blocked groups" + ); + // Safety: `block_size` is always set in blocked groups + let block_size = self.block_size.unwrap(); + *num_values = num_values.saturating_sub(block_size); + None + } + }, + SeenValues::Some { builder } => Some(builder.emit(emit_to)), + } + } + + #[cfg(test)] + fn num_blocks(&self) -> usize { + match &self.seen_values { + SeenValues::All { .. } => 0, + SeenValues::Some { builder } => builder.num_blocks(), + } + } +} + +/// Adapter for supporting dynamic dispatching of [`FlatNullState`] and [`BlockedNullState`]. +/// For performance, the cost of batch-level dynamic dispatching is acceptable. +#[derive(Debug)] +pub enum NullStateAdapter { + Flat(FlatNullState), + Blocked(BlockedNullState), +} + +impl NullStateAdapter { + pub fn new(block_size: Option) -> Self { + if block_size.is_some() { + Self::Blocked(BlockedNullState::new(block_size)) + } else { + Self::Flat(FlatNullState::new(block_size)) + } + } + + pub fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + value_fn: F, + ) where + T: ArrowPrimitiveType + Send, + F: FnMut(u32, u64, T::Native) + Send, + { + match self { + NullStateAdapter::Flat(null_state) => null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ), + NullStateAdapter::Blocked(null_state) => null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ), + } + } + + pub fn accumulate_boolean( + &mut self, + group_indices: &[usize], + values: &BooleanArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + value_fn: F, + ) where + F: FnMut(u32, u64, bool) + Send, + { + match self { + NullStateAdapter::Flat(null_state) => null_state.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ), + NullStateAdapter::Blocked(null_state) => null_state.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ), + } + } + + pub fn build(&mut self, emit_to: EmitTo) -> Option { + match self { + NullStateAdapter::Flat(null_state) => null_state.build(emit_to), + NullStateAdapter::Blocked(null_state) => null_state.build(emit_to), + } + } + + pub fn size(&self) -> usize { + match self { + NullStateAdapter::Flat(null_state) => null_state.size(), + NullStateAdapter::Blocked(null_state) => null_state.size(), + } + } + + /// Clone and build a single [`BooleanBuffer`] from `seen_values`, + /// only used for testing. + #[cfg(test)] + fn build_cloned_seen_values(&self) -> TestSeenValuesResult { + match self { + NullStateAdapter::Flat(null_state) => match &null_state.seen_values { + SeenValues::All { num_values } => TestSeenValuesResult::All(*num_values), + SeenValues::Some { builder } => { + let seen_values = builder[0].finish_cloned(); + TestSeenValuesResult::Some(seen_values) + } + }, + NullStateAdapter::Blocked(null_state) => match &null_state.seen_values { + SeenValues::All { num_values } => TestSeenValuesResult::All(*num_values), + SeenValues::Some { builder } => { + let mut return_builder = BooleanBufferBuilder::new(0); + let num_blocks = builder.num_blocks(); + for blk_idx in 0..num_blocks { + let bool_builder = &builder[blk_idx]; + for idx in 0..bool_builder.len() { + return_builder.append(bool_builder.get_bit(idx)); + } + } + let seen_values = return_builder.finish(); + TestSeenValuesResult::Some(seen_values) } }, } } + + #[cfg(test)] + fn build_single_null_buffer(&mut self) -> Option { + match self { + NullStateAdapter::Flat(null_state) => null_state.build(EmitTo::All), + NullStateAdapter::Blocked(null_state) => { + let mut return_builder = BooleanBufferBuilder::new(0); + let num_blocks = null_state.num_blocks(); + let mut has_some_nulls = false; + for _ in 0..num_blocks { + let blocked_nulls = null_state.build(EmitTo::NextBlock); + if let Some(nulls) = blocked_nulls { + has_some_nulls = true; + for bit in nulls.inner().iter() { + return_builder.append(bit); + } + } + } + + if has_some_nulls { + Some(NullBuffer::new(return_builder.finish())) + } else { + None + } + } + } + } +} + +#[cfg(test)] +enum TestSeenValuesResult { + All(usize), + Some(BooleanBuffer), +} + +/// [`NullState`] for `flat groups input` +/// +/// At first, you may need to see something about `block_id` and `block_offset` +/// from [`GroupsAccumulator::supports_blocked_groups`]. +/// +/// The `flat groups input` are organized like: +/// +/// ```text +/// row_0 group_index_0 +/// row_1 group_index_1 +/// row_2 group_index_2 +/// ... +/// row_n group_index_n +/// ``` +/// +/// If `row_x group_index_x` is not filtered(`group_index_x` is seen) +/// `seen_values[group_index_x]` will be set to `true`. +/// +/// For `set_bit(block_id, block_offset, value)`, `block_id` is unused, +/// `block_offset` will be set to `group_index`. +/// +/// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator::supports_blocked_groups +/// +pub type FlatNullState = NullState; + +/// [`NullState`] for `blocked groups input` +/// +/// At first, you may need to see something about `block_id` and `block_offset` +/// from [`GroupsAccumulator::supports_blocked_groups`]. +/// +/// The `flat groups input` are organized like: +/// +/// ```text +/// row_0 (block_id_0, block_offset_0) +/// row_1 (block_id_1, block_offset_1) +/// row_2 (block_id_1, block_offset_1) +/// ... +/// row_n (block_id_n, block_offset_n) +/// ``` +/// +/// If `row_x (block_id_x, block_offset_x)` is not filtered +/// (`block_id_x, block_offset_x` is seen), `seen_values[block_id_x][block_offset_x]` +/// will be set to `true`. +/// +/// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator::supports_blocked_groups +/// +pub type BlockedNullState = NullState; + +impl Blocks { + fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool) { + self[block_id as usize].set_bit(block_offset as usize, value); + } + + fn size(&self) -> usize { + if self.is_empty() { + return 0; + } + self[0].capacity() / 8 * self.num_blocks() + } + + fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { + let nulls = match emit_to { + EmitTo::All | EmitTo::First(_) => self[0].finish(), + EmitTo::NextBlock => { + let mut block = self + .pop_block() + .expect("should not try to emit empty blocks"); + block.finish() + } + }; + + let nulls = if let EmitTo::First(n) = emit_to { + // split off the first N values in seen_values + // + // TODO make this more efficient rather than two + // copies and bitwise manipulation + let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); + // reset the existing seen buffer + for seen in nulls.iter().skip(n) { + self[0].append(seen); + } + first_n_null + } else { + nulls + }; + + NullBuffer::new(nulls) + } } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -482,6 +819,8 @@ pub fn accumulate( /// * `group_idx`: The group index for the current row /// * `batch_idx`: The index of the current row in the input arrays /// * `columns`: Reference to all input arrays for accessing values +// TODO: support `blocked group index` for `accumulate_multiple` +// (for supporting `blocked group index` for correlation group accumulator) pub fn accumulate_multiple( group_indices: &[usize], value_columns: &[&PrimitiveArray], @@ -545,6 +884,8 @@ pub fn accumulate_multiple( /// /// See [`NullState::accumulate`], for more details on other /// arguments. +// TODO: support `blocked group index` for `accumulate_indices` +// (for supporting `blocked group index` for count group accumulator) pub fn accumulate_indices( group_indices: &[usize], nulls: Option<&NullBuffer>, @@ -682,41 +1023,19 @@ mod test { buffer::BooleanBuffer, }; use rand::{Rng, rngs::ThreadRng}; - use std::collections::HashSet; + use std::{cmp, collections::HashMap, collections::HashSet}; + /// Tests NullState accumulation with a deterministic fixture: 100 dense + /// group indices (0..100), fixed null/filter patterns, block_size=3, + /// acc_rounds=5. Covers flat and blocked modes, with/without nulls and filters. #[test] - fn accumulate() { - let group_indices = (0..100).collect(); - let values = (0..100).map(|i| (i + 1) * 10).collect(); - let values_with_nulls = (0..100) - .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) - .collect(); - - // default to every fifth value being false, every even - // being null - let filter: BooleanArray = (0..100) - .map(|i| { - let is_even = i % 2 == 0; - let is_fifth = i % 5 == 0; - if is_even { - None - } else if is_fifth { - Some(false) - } else { - Some(true) - } - }) - .collect(); - - Fixture { - group_indices, - values, - values_with_nulls, - filter, - } - .run() + fn accumulate_fixed() { + Fixture::new_fixed().run(); } + /// Tests NullState accumulation with 100 randomized fixtures. + /// Group indices are generated via hash-table-style dedup (dense 0..N), + /// matching real GroupedHashAggregateStream behavior. #[test] fn accumulate_fuzz() { let mut rng = rand::rng(); @@ -739,22 +1058,86 @@ mod test { /// filter (defaults to None) filter: BooleanArray, + + /// block size for testing [`BlockedNullState`] + block_size: usize, + + acc_rounds: usize, } impl Fixture { + fn new_fixed() -> Self { + let group_indices = (0..100).collect(); + let values = (0..100).map(|i| (i + 1) * 10).collect(); + let values_with_nulls = (0..100) + .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) + .collect(); + + // default to every fifth value being false, every even + // being null + let filter: BooleanArray = (0..100) + .map(|i| { + let is_even = i % 2 == 0; + let is_fifth = i % 5 == 0; + if is_even { + None + } else if is_fifth { + Some(false) + } else { + Some(true) + } + }) + .collect(); + + Self { + group_indices, + values, + values_with_nulls, + filter, + block_size: 3, + acc_rounds: 5, + } + } + fn new_random(rng: &mut ThreadRng) -> Self { // Number of input values in a batch let num_values: usize = rng.random_range(1..200); - // number of distinct groups - let num_groups: usize = rng.random_range(2..1000); - let max_group = num_groups - 1; - + // number of distinct "raw keys" each row can map to + let num_distinct_keys: usize = rng.random_range(2..1000); + + // Simulate hash-table dedup: each row gets a random raw key, + // and a HashMap assigns group indices sequentially (0, 1, 2, …). + // This guarantees group_indices are dense in 0..num_groups, just + // like real GroupedHashAggregateStream behavior. + // + // Example: raw keys = [7, 3, 7, 5, 3] + // → first-seen order: 7→0, 3→1, 5→2 + // → group_indices : [0, 1, 0, 2, 1] + // → total_num_groups = 3 (all of 0,1,2 actually appeared) + let mut key_to_group: HashMap = HashMap::new(); + let mut next_group_index: usize = 0; let group_indices: Vec = (0..num_values) - .map(|_| rng.random_range(0..max_group)) + .map(|_| { + let raw_key = rng.random_range(0..num_distinct_keys); + let len = key_to_group.len(); + *key_to_group.entry(raw_key).or_insert_with(|| { + let idx = len; + next_group_index = len + 1; + idx + }) + }) .collect(); + let num_groups = next_group_index; + let values: Vec = (0..num_values).map(|_| rng.random()).collect(); + // random block size + let block_size = rng.random_range(1..cmp::max(num_groups, 2)); + + // random acc rounds + let acc_rounds = rng.random_range(1..=group_indices.len()); + // 10% chance of false // 10% change of null // 80% chance of true @@ -786,6 +1169,8 @@ mod test { values, values_with_nulls, filter, + block_size, + acc_rounds, } } @@ -810,7 +1195,14 @@ mod test { let filter = &self.filter; // no null, no filters - Self::accumulate_test(group_indices, &values_array, None, total_num_groups); + Self::accumulate_test( + group_indices, + &values_array, + None, + total_num_groups, + self.block_size, + self.acc_rounds, + ); // nulls, no filters Self::accumulate_test( @@ -818,6 +1210,8 @@ mod test { &values_with_nulls_array, None, total_num_groups, + self.block_size, + self.acc_rounds, ); // no nulls, filters @@ -826,6 +1220,8 @@ mod test { &values_array, Some(filter), total_num_groups, + self.block_size, + self.acc_rounds, ); // nulls, filters @@ -834,6 +1230,8 @@ mod test { &values_with_nulls_array, Some(filter), total_num_groups, + self.block_size, + self.acc_rounds, ); } @@ -844,26 +1242,97 @@ mod test { values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + block_size: usize, + acc_rounds: usize, ) { + // Test `accumulate` of `FlatNullState` + accumulate in once Self::accumulate_values_test( group_indices, values, opt_filter, total_num_groups, + None, + None, + ); + + // Test `accumulate` of `FlatNullState` + accumulate in multiple times + Self::accumulate_values_test( + group_indices, + values, + opt_filter, + total_num_groups, + None, + Some(acc_rounds), + ); + + // Test `accumulate` of `BlockedNullState` + accumulate in once + Self::accumulate_values_test( + group_indices, + values, + opt_filter, + total_num_groups, + Some(block_size), + None, + ); + + // Test `accumulate` of `BlockedNullState` + accumulate in multiple times + Self::accumulate_values_test( + group_indices, + values, + opt_filter, + total_num_groups, + Some(block_size), + Some(acc_rounds), ); - Self::accumulate_indices_test(group_indices, values.nulls(), opt_filter); // Convert values into a boolean array (anything above the // average is true, otherwise false) let avg: usize = values.iter().filter_map(|v| v.map(|v| v as usize)).sum(); let boolean_values: BooleanArray = values.iter().map(|v| v.map(|v| v as usize > avg)).collect(); + + // Test `accumulate_boolean` of `FlatNullState` + accumulate in once Self::accumulate_boolean_test( group_indices, &boolean_values, opt_filter, total_num_groups, + None, + None, + ); + + // Test `accumulate_boolean` of `FlatNullState` + accumulate in multiple times + Self::accumulate_boolean_test( + group_indices, + &boolean_values, + opt_filter, + total_num_groups, + None, + Some(acc_rounds), + ); + + // Test `accumulate_boolean` of `BlockedNullState` + accumulate in once + Self::accumulate_boolean_test( + group_indices, + &boolean_values, + opt_filter, + total_num_groups, + Some(block_size), + None, + ); + + // Test `accumulate_boolean` of `BlockedNullState` + accumulate in multiple times + Self::accumulate_boolean_test( + group_indices, + &boolean_values, + opt_filter, + total_num_groups, + Some(block_size), + Some(acc_rounds), ); + + // Test `accumulate_indices` + Self::accumulate_indices_test(group_indices, values.nulls(), opt_filter); } /// This is effectively a different implementation of @@ -873,19 +1342,105 @@ mod test { values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + block_size: Option, + acc_rounds: Option, ) { - let mut accumulated_values = vec![]; - let mut null_state = NullState::new(); + // Chunking `group_indices`, `values`, `opt_filter`, and we also need to generate + // `chunked acc_group_indices` basing on `group_indices` + let (group_indices_chunks, values_chunks, opt_filter_chunks) = + if let Some(rounds) = acc_rounds { + let chunk_size = group_indices.len() / rounds; + + let group_indices_chunks = group_indices + .chunks(chunk_size) + .map(|chunk| chunk.to_vec()) + .collect::>(); + + let values_chunks = values + .iter() + .collect::>() + .chunks(chunk_size) + .map(|chunk| UInt32Array::from_iter(chunk.iter().copied())) + .collect::>(); + + let opt_filter_chunks = if let Some(filter) = opt_filter { + filter + .iter() + .collect::>() + .chunks(chunk_size) + .map(|chunk| Some(BooleanArray::from_iter(chunk.iter()))) + .collect::>() + } else { + vec![None; values_chunks.len()] + }; - null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); - }, - ); + (group_indices_chunks, values_chunks, opt_filter_chunks) + } else { + ( + vec![group_indices.to_vec()], + vec![values.clone()], + vec![opt_filter.cloned()], + ) + }; + + let mut total_num_groups_chunks = vec![]; + let mut cur_total_num_groups = usize::MIN; + for group_indices in &group_indices_chunks { + let num_groups = *group_indices.iter().max().unwrap() + 1; + cur_total_num_groups = cmp::max(cur_total_num_groups, num_groups); + total_num_groups_chunks.push(cur_total_num_groups); + } + + // Build needed test contexts + let (mut null_state, block_size, acc_group_indices_chunks) = + if let Some(blk_size) = block_size { + let mut acc_group_indices_chunks = vec![]; + for group_indices in group_indices_chunks { + let acc_group_indices = group_indices + .into_iter() + .map(|index| { + let block_id = (index / blk_size) as u32; + let block_offset = (index % blk_size) as u64; + BlockedGroupIndexOperations::pack_index( + block_id, + block_offset, + ) as usize + }) + .collect::>(); + acc_group_indices_chunks.push(acc_group_indices); + } + + ( + NullStateAdapter::new(Some(blk_size)), + blk_size, + acc_group_indices_chunks, + ) + } else { + (NullStateAdapter::new(None), 0, group_indices_chunks) + }; + + // Start the test + let mut accumulated_values = vec![]; + for (((acc_group_indices, values), total_num_groups), cur_opt_filter) in + acc_group_indices_chunks + .into_iter() + .zip(values_chunks) + .zip(total_num_groups_chunks) + .zip(opt_filter_chunks) + { + null_state.accumulate( + &acc_group_indices, + &values, + cur_opt_filter.as_ref(), + total_num_groups, + |block_id, block_offset, value| { + let flatten_index = ((block_id as u64 * block_size as u64) + + block_offset) + as usize; + accumulated_values.push((flatten_index, value)); + }, + ); + } // Figure out the expected values let mut expected_values = vec![]; @@ -921,21 +1476,20 @@ mod test { accumulated_values, expected_values, "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}" ); - - match &null_state.seen_values { - SeenValues::All { num_values } => { - assert_eq!(*num_values, total_num_groups); + let seen_values_result = null_state.build_cloned_seen_values(); + match &seen_values_result { + TestSeenValuesResult::All(num_values) => { + assert_eq!(*num_values, total_num_groups) } - SeenValues::Some { values } => { - let seen_values = values.finish_cloned(); - mock.validate_seen_values(&seen_values); + TestSeenValuesResult::Some(seen_values) => { + mock.validate_seen_values(seen_values); } } // Validate the final buffer (one value per group) let expected_null_buffer = mock.expected_null_buffer(total_num_groups); - let null_buffer = null_state.build(EmitTo::All); + let null_buffer = null_state.build_single_null_buffer(); if let Some(nulls) = &null_buffer { assert_eq!(*nulls, expected_null_buffer); } @@ -1002,19 +1556,105 @@ mod test { values: &BooleanArray, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + block_size: Option, + acc_rounds: Option, ) { - let mut accumulated_values = vec![]; - let mut null_state = NullState::new(); + // Chunking `group_indices`, `values`, `opt_filter`, and we also need to generate + // `chunked acc_group_indices` basing on `group_indices` + let (group_indices_chunks, values_chunks, opt_filter_chunks) = + if let Some(rounds) = acc_rounds { + let chunk_size = group_indices.len() / rounds; + + let group_indices_chunks = group_indices + .chunks(chunk_size) + .map(|chunk| chunk.to_vec()) + .collect::>(); + + let values_chunks = values + .iter() + .collect::>() + .chunks(chunk_size) + .map(|chunk| BooleanArray::from_iter(chunk.iter().copied())) + .collect::>(); + + let opt_filter_chunks = if let Some(filter) = opt_filter { + filter + .iter() + .collect::>() + .chunks(chunk_size) + .map(|chunk| Some(BooleanArray::from_iter(chunk.iter()))) + .collect::>() + } else { + vec![None; values_chunks.len()] + }; - null_state.accumulate_boolean( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); - }, - ); + (group_indices_chunks, values_chunks, opt_filter_chunks) + } else { + ( + vec![group_indices.to_vec()], + vec![values.clone()], + vec![opt_filter.cloned()], + ) + }; + + let mut total_num_groups_chunks = vec![]; + let mut cur_total_num_groups = usize::MIN; + for group_indices in &group_indices_chunks { + let num_groups = *group_indices.iter().max().unwrap() + 1; + cur_total_num_groups = cmp::max(cur_total_num_groups, num_groups); + total_num_groups_chunks.push(cur_total_num_groups); + } + + // Build needed test contexts + let (mut null_state, block_size, acc_group_indices_chunks) = + if let Some(blk_size) = block_size { + let mut acc_group_indices_chunks = vec![]; + for group_indices in group_indices_chunks { + let acc_group_indices = group_indices + .into_iter() + .map(|index| { + let block_id = (index / blk_size) as u32; + let block_offset = (index % blk_size) as u64; + BlockedGroupIndexOperations::pack_index( + block_id, + block_offset, + ) as usize + }) + .collect::>(); + acc_group_indices_chunks.push(acc_group_indices); + } + + ( + NullStateAdapter::new(Some(blk_size)), + blk_size, + acc_group_indices_chunks, + ) + } else { + (NullStateAdapter::new(None), 0, group_indices_chunks) + }; + + // Start the test + let mut accumulated_values = vec![]; + for (((acc_group_indices, values), total_num_groups), opt_filter) in + acc_group_indices_chunks + .into_iter() + .zip(values_chunks) + .zip(total_num_groups_chunks) + .zip(opt_filter_chunks) + { + null_state.accumulate_boolean( + &acc_group_indices, + &values, + opt_filter.as_ref(), + total_num_groups, + |block_id, block_offset, value| { + let flatten_index = ((block_id as u64 * block_size as u64) + + block_offset) + as usize; + accumulated_values.push((flatten_index, value)); + }, + ); + } // Figure out the expected values let mut expected_values = vec![]; @@ -1051,24 +1691,22 @@ mod test { "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}" ); - match &null_state.seen_values { - SeenValues::All { num_values } => { - assert_eq!(*num_values, total_num_groups); + let seen_values_result = null_state.build_cloned_seen_values(); + match &seen_values_result { + TestSeenValuesResult::All(num_values) => { + assert_eq!(*num_values, total_num_groups) } - SeenValues::Some { values } => { - let seen_values = values.finish_cloned(); - mock.validate_seen_values(&seen_values); + TestSeenValuesResult::Some(seen_values) => { + mock.validate_seen_values(seen_values); } } // Validate the final buffer (one value per group) - let expected_null_buffer = Some(mock.expected_null_buffer(total_num_groups)); - - let is_all_seen = matches!(null_state.seen_values, SeenValues::All { .. }); - let null_buffer = null_state.build(EmitTo::All); + let expected_null_buffer = mock.expected_null_buffer(total_num_groups); + let null_buffer = null_state.build_single_null_buffer(); - if !is_all_seen { - assert_eq!(null_buffer, expected_null_buffer); + if let Some(buffer) = null_buffer { + assert_eq!(buffer, expected_null_buffer); } } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs new file mode 100644 index 0000000000000..c1380376896ce --- /dev/null +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -0,0 +1,364 @@ +// 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. + +//! Aggregation intermediate results blocks in blocked approach + +use std::{ + fmt::Debug, + iter, mem, + ops::{Index, IndexMut}, +}; + +use datafusion_expr_common::groups_accumulator::EmitTo; + +/// Structure used to store aggregation intermediate results in `blocked approach` +/// +/// Aggregation intermediate results will be stored as multiple [`Block`]s +/// (simply you can think a [`Block`] as a `Vec`). And `Blocks` is the structure +/// to represent such multiple [`Block`]s. +/// +/// Internally uses a `Vec` with a `start` offset to track the first active +/// block. When blocks are popped via [`Self::pop_block`], the block is swapped +/// out in O(1) using `mem::replace` and the `start` cursor advances, avoiding +/// the O(n) shift cost of `Vec::remove(0)`. +/// +/// More details about `blocked approach` can see in: [`GroupsAccumulator::supports_blocked_groups`]. +/// +/// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator::supports_blocked_groups +/// +#[derive(Debug)] +pub struct Blocks { + inner: Vec, + /// Index of the first active block. All blocks before this index have been + /// popped and replaced with empty placeholders. + start: usize, + block_size: Option, +} + +impl Blocks { + pub fn new(block_size: Option) -> Self { + Self { + inner: Vec::new(), + start: 0, + block_size, + } + } + + pub fn resize( + &mut self, + total_num_groups: usize, + new_block: F, + default_value: B::T, + ) where + F: Fn(Option) -> B, + { + let block_size = self.block_size.unwrap_or(usize::MAX); + // For resize, we need to: + // 1. Ensure the blks are enough first + // 2. and then ensure slots in blks are enough + let active_len = self.inner.len() - self.start; + let (mut cur_blk_idx, exist_slots_num) = if active_len > 0 { + let cur_blk_idx = self.inner.len() - 1; + let exist_slots = + (active_len - 1) * block_size + self.inner.last().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (self.inner.len(), 0) + }; + + // No new groups, don't need to expand, just return + if exist_slots_num >= total_num_groups { + return; + } + + // 1. Ensure blks are enough + let new_blks_num = total_num_groups.div_ceil(block_size) - active_len; + if new_blks_num > 0 { + for _ in 0..new_blks_num { + let block = new_block(self.block_size); + self.inner.push(block); + } + } + + // 2. Ensure slots are enough + let mut new_slots_num = total_num_groups - exist_slots_num; + + // When no active blocks existed before, cur_blk_idx now points to the + // first newly pushed block. + if active_len == 0 { + cur_blk_idx = self.start; + } + + // 2.1 Only fill current blk if it may be already enough + let cur_blk_rest_slots_num = block_size - self.inner[cur_blk_idx].len(); + if cur_blk_rest_slots_num >= new_slots_num { + self.inner[cur_blk_idx] + .fill_default_value(new_slots_num, default_value.clone()); + return; + } + + // 2.2 Fill current blk to full + self.inner[cur_blk_idx] + .fill_default_value(cur_blk_rest_slots_num, default_value.clone()); + new_slots_num -= cur_blk_rest_slots_num; + + // 2.3 Fill complete blks + let complete_blks_num = new_slots_num / block_size; + for _ in 0..complete_blks_num { + cur_blk_idx += 1; + self.inner[cur_blk_idx].fill_default_value(block_size, default_value.clone()); + } + + // 2.4 Fill last blk if needed + let rest_slots_num = new_slots_num % block_size; + if rest_slots_num > 0 { + self.inner + .last_mut() + .unwrap() + .fill_default_value(rest_slots_num, default_value); + } + } + + /// Pop the first active block in O(1) by swapping it with an empty + /// placeholder and advancing the `start` cursor. + pub fn pop_block(&mut self) -> Option { + if self.start >= self.inner.len() { + None + } else { + let block = mem::take(&mut self.inner[self.start]); + self.start += 1; + Some(block) + } + } + + pub fn num_blocks(&self) -> usize { + self.inner.len() - self.start + } + + pub fn is_empty(&self) -> bool { + self.start >= self.inner.len() + } + + pub fn iter(&self) -> impl Iterator { + self.inner[self.start..].iter() + } + + pub fn clear(&mut self) { + self.inner.clear(); + self.start = 0; + } +} + +impl Index for Blocks { + type Output = B; + + #[inline] + fn index(&self, index: usize) -> &Self::Output { + &self.inner[self.start + index] + } +} + +impl IndexMut for Blocks { + #[inline] + fn index_mut(&mut self, index: usize) -> &mut Self::Output { + &mut self.inner[self.start + index] + } +} + +/// The abstraction to represent one aggregation intermediate result block +/// in `blocked approach`, multiple blocks compose a [`Blocks`] +/// +/// Many types of aggregation intermediate result exist, and we define an interface +/// to abstract the necessary behaviors of various intermediate result types. +/// +pub trait Block: Debug + Default { + type T: Clone; + + /// Fill the block with default value + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); + + /// Return the length of the block + fn len(&self) -> usize; + + /// Return true if the block is empty + fn is_empty(&self) -> bool { + self.len() == 0 + } +} + +/// Usually we use `Vec` to represent `Block`, so we define `Blocks>` +/// as the `VecBlocks` +pub type VecBlocks = Blocks>; + +/// We usually use `Vec` to represent `Block`, +/// so we implement `Block` trait for `Vec` +impl Block for Vec { + type T = Ty; + + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { + self.extend(iter::repeat_n(default_value, fill_len)); + } + + fn len(&self) -> usize { + self.len() + } +} + +impl VecBlocks { + pub fn emit(&mut self, emit_to: EmitTo) -> Vec { + if matches!(emit_to, EmitTo::NextBlock) { + assert!( + self.block_size.is_some(), + "only support emit next block in blocked groups" + ); + self.pop_block() + .expect("should not call emit for empty blocks") + } else { + // TODO: maybe remove `EmitTo::take_needed` and move the + // pattern matching codes here after supporting blocked approach + // for all exist accumulators, to avoid matching twice + assert!( + self.block_size.is_none(), + "only support emit all/first in flat groups" + ); + emit_to.take_needed(&mut self[0]) + } + } +} + +#[cfg(test)] +mod test { + use crate::aggregate::groups_accumulator::blocks::Blocks; + + type TestBlocks = Blocks>; + + #[test] + fn test_single_block_resize() { + let new_block = |block_size: Option| { + let cap = block_size.unwrap_or(0); + Vec::with_capacity(cap) + }; + + let mut blocks = TestBlocks::new(None); + assert_eq!(blocks.num_blocks(), 0); + + for _ in 0..2 { + // Should have single block, 5 block len, all data are 42 + blocks.resize(5, new_block, 42); + assert_eq!(blocks.num_blocks(), 1); + assert_eq!(blocks[0].len(), 5); + blocks[0].iter().for_each(|num| assert_eq!(*num, 42)); + + // Resize to a larger block + // Should still have single block, 10 block len, all data are 42 + blocks.resize(10, new_block, 42); + assert_eq!(blocks.num_blocks(), 1); + assert_eq!(blocks[0].len(), 10); + blocks[0].iter().for_each(|num| assert_eq!(*num, 42)); + + // Clear + // Should have nothing after clearing + blocks.clear(); + assert_eq!(blocks.num_blocks(), 0); + + // Test resize after clear in next round + } + } + + #[test] + fn test_multi_blocks_resize() { + let new_block = |block_size: Option| { + let cap = block_size.unwrap_or(0); + Vec::with_capacity(cap) + }; + + let mut blocks = TestBlocks::new(Some(3)); + assert_eq!(blocks.num_blocks(), 0); + + for _ in 0..2 { + // Should have: + // - 2 blocks + // - `block 0` of 3 len + // - `block 1` of 2 len + // - all data are 42 + blocks.resize(5, new_block, 42); + assert_eq!(blocks.num_blocks(), 2); + assert_eq!(blocks[0].len(), 3); + blocks[0].iter().for_each(|num| assert_eq!(*num, 42)); + assert_eq!(blocks[1].len(), 2); + blocks[1].iter().for_each(|num| assert_eq!(*num, 42)); + + // Resize to larger blocks + // Should have: + // - 4 blocks + // - `block 0` of 3 len + // - `block 1` of 3 len + // - `block 2` of 3 len + // - `block 3` of 1 len + // - all data are 42 + blocks.resize(10, new_block, 42); + assert_eq!(blocks.num_blocks(), 4); + assert_eq!(blocks[0].len(), 3); + blocks[0].iter().for_each(|num| assert_eq!(*num, 42)); + assert_eq!(blocks[1].len(), 3); + blocks[1].iter().for_each(|num| assert_eq!(*num, 42)); + assert_eq!(blocks[2].len(), 3); + blocks[2].iter().for_each(|num| assert_eq!(*num, 42)); + assert_eq!(blocks[3].len(), 1); + blocks[3].iter().for_each(|num| assert_eq!(*num, 42)); + + // Clear + // Should have nothing after clearing + blocks.clear(); + assert_eq!(blocks.num_blocks(), 0); + + // Test resize after clear in next round + } + } + + #[test] + fn test_pop_block() { + let new_block = |block_size: Option| { + let cap = block_size.unwrap_or(0); + Vec::with_capacity(cap) + }; + + let mut blocks = TestBlocks::new(Some(3)); + blocks.resize(7, new_block, 42); + // 3 blocks: [42,42,42], [42,42,42], [42] + assert_eq!(blocks.num_blocks(), 3); + + let blk0 = blocks.pop_block().unwrap(); + assert_eq!(blk0.len(), 3); + assert_eq!(blocks.num_blocks(), 2); + // blocks[0] should now be what was blocks[1] + assert_eq!(blocks[0].len(), 3); + assert_eq!(blocks[1].len(), 1); + + let blk1 = blocks.pop_block().unwrap(); + assert_eq!(blk1.len(), 3); + assert_eq!(blocks.num_blocks(), 1); + assert_eq!(blocks[0].len(), 1); + + let blk2 = blocks.pop_block().unwrap(); + assert_eq!(blk2.len(), 1); + assert_eq!(blocks.num_blocks(), 0); + assert!(blocks.is_empty()); + assert!(blocks.pop_block().is_none()); + } +} diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs index d1d8924a2c3e8..cc1ccd14c11fa 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/bool_op.rs @@ -20,10 +20,10 @@ use std::sync::Arc; use crate::aggregate::groups_accumulator::nulls::filtered_null_mask; use arrow::array::{ArrayRef, AsArray, BooleanArray, BooleanBufferBuilder}; use arrow::buffer::BooleanBuffer; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -use super::accumulate::NullState; +use super::accumulate::FlatNullState; /// An accumulator that implements a single operation over a /// [`BooleanArray`] where the accumulated state is also boolean (such @@ -43,7 +43,7 @@ where values: BooleanBufferBuilder, /// Track nulls in the input / filters - null_state: NullState, + null_state: FlatNullState, /// Function that computes the output bool_fn: F, @@ -60,7 +60,7 @@ where pub fn new(bool_fn: F, identity: bool) -> Self { Self { values: BooleanBufferBuilder::new(0), - null_state: NullState::new(), + null_state: FlatNullState::new(None), bool_fn, identity, } @@ -94,7 +94,8 @@ where values, opt_filter, total_num_groups, - |group_index, new_value| { + |_, group_index, new_value| { + let group_index = group_index as usize; let current_value = self.values.get_bit(group_index); let value = (self.bool_fn)(current_value, new_value); self.values.set_bit(group_index, value); @@ -117,6 +118,9 @@ where } first_n } + EmitTo::NextBlock => { + return internal_err!("boolean_op does not support blocked groups"); + } }; let nulls = self.null_state.build(emit_to); diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/group_index_operations.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/group_index_operations.rs new file mode 100644 index 0000000000000..a62b21fa1be14 --- /dev/null +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/group_index_operations.rs @@ -0,0 +1,91 @@ +// 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. + +//! Useful tools for operating group index + +use std::fmt::Debug; + +/// Operations about group index parsing +/// +/// There are mainly 2 `group index` needing parsing: `flat` and `blocked`. +/// +/// # Flat group index +/// `flat group index` format is like: +/// +/// ```text +/// | block_offset(64bit) | +/// ``` +/// +/// It is used in `flat GroupValues/GroupAccumulator`, only a single block +/// exists, so its `block_id` is always 0, and use all 64 bits to store the +/// `block offset`. +/// +/// # Blocked group index +/// `blocked group index` format is like: +/// +/// ```text +/// | block_id(32bit) | block_offset(32bit) +/// ``` +/// +/// It is used in `blocked GroupValues/GroupAccumulator`, multiple blocks +/// exist, and we use high 32 bits to store `block_id`, and low 32 bit to +/// store `block_offset`. +/// +/// The `get_block_offset` method requires to return `block_offset` as u64, +/// that is for compatible for `flat group index`'s parsing. +/// +pub trait GroupIndexOperations: Debug { + fn pack_index(block_id: u32, block_offset: u64) -> u64; + + fn get_block_id(packed_index: u64) -> u32; + + fn get_block_offset(packed_index: u64) -> u64; +} + +#[derive(Debug)] +pub struct BlockedGroupIndexOperations; + +impl GroupIndexOperations for BlockedGroupIndexOperations { + fn pack_index(block_id: u32, block_offset: u64) -> u64 { + ((block_id as u64) << 32) | block_offset + } + + fn get_block_id(packed_index: u64) -> u32 { + (packed_index >> 32) as u32 + } + + fn get_block_offset(packed_index: u64) -> u64 { + (packed_index as u32) as u64 + } +} + +#[derive(Debug)] +pub struct FlatGroupIndexOperations; + +impl GroupIndexOperations for FlatGroupIndexOperations { + fn pack_index(_block_id: u32, block_offset: u64) -> u64 { + block_offset + } + + fn get_block_id(_packed_index: u64) -> u32 { + 0 + } + + fn get_block_offset(packed_index: u64) -> u64 { + packed_index + } +} diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs index a81b89e1e46f1..665799a980f52 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/prim_op.rs @@ -26,7 +26,8 @@ use arrow::datatypes::DataType; use datafusion_common::{DataFusionError, Result, internal_datafusion_err}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -use super::accumulate::NullState; +use crate::aggregate::groups_accumulator::accumulate::NullStateAdapter; +use crate::aggregate::groups_accumulator::blocks::{Blocks, VecBlocks}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -43,8 +44,8 @@ where T: ArrowPrimitiveType + Send, F: Fn(&mut T::Native, T::Native) + Send + Sync + 'static, { - /// values per group, stored as the native type - values: Vec, + /// Values per group, stored as the native type + values: VecBlocks, /// The output type (needed for Decimal precision and scale) data_type: DataType, @@ -53,7 +54,7 @@ where starting_value: T::Native, /// Track nulls in the input / filters - null_state: NullState, + null_state: NullStateAdapter, /// Function that computes the primitive result prim_fn: F, @@ -66,9 +67,9 @@ where { pub fn new(data_type: &DataType, prim_fn: F) -> Self { Self { - values: vec![], + values: Blocks::new(None), data_type: data_type.clone(), - null_state: NullState::new(), + null_state: NullStateAdapter::new(None), starting_value: T::default_value(), prim_fn, } @@ -96,8 +97,19 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); - // update values - self.values.resize(total_num_groups, self.starting_value); + // Expand to ensure values are large enough + let new_block = |block_size: Option| { + // In blocked mode, pre-allocate the full block capacity. + // In flat mode (block_size=None), start with an empty Vec + // and let `resize` grow it to exactly `total_num_groups`, + // matching the standard Vec growth behavior. + match block_size { + Some(cap) => Vec::with_capacity(cap), + None => Vec::new(), + } + }; + self.values + .resize(total_num_groups, new_block, self.starting_value); // NullState dispatches / handles tracking nulls and groups that saw no values self.null_state.accumulate( @@ -105,9 +117,12 @@ where values, opt_filter, total_num_groups, - |group_index, new_value| { - // SAFETY: group_index is guaranteed to be in bounds - let value = unsafe { self.values.get_unchecked_mut(group_index) }; + |block_id, block_offset, new_value| { + // SAFETY: `block_id` and `block_offset` are guaranteed to be in bounds + let value = unsafe { + self.values[block_id as usize] + .get_unchecked_mut(block_offset as usize) + }; (self.prim_fn)(value, new_value); }, ); @@ -116,7 +131,7 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = emit_to.take_needed(&mut self.values); + let values = self.values.emit(emit_to); let nulls = self.null_state.build(emit_to); let values = PrimitiveArray::::new(values.into(), nulls) // no copy .with_data_type(self.data_type.clone()); @@ -196,6 +211,24 @@ where } fn size(&self) -> usize { - self.values.capacity() * size_of::() + self.null_state.size() + if self.values.is_empty() { + return 0; + } + let values_cap = self.values.num_blocks() * self.values[0].capacity(); + let values_size = values_cap * size_of::(); + let nulls_size = self.null_state.size(); + values_size + nulls_size + } + + fn supports_blocked_groups(&self) -> bool { + true + } + + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { + self.values.clear(); + self.values = Blocks::new(block_size); + self.null_state = NullStateAdapter::new(block_size); + + Ok(()) } } diff --git a/datafusion/functions-aggregate/src/array_agg.rs b/datafusion/functions-aggregate/src/array_agg.rs index 861d7712ba1b0..c777c3f979bed 100644 --- a/datafusion/functions-aggregate/src/array_agg.rs +++ b/datafusion/functions-aggregate/src/array_agg.rs @@ -34,7 +34,9 @@ use datafusion_common::cast::as_list_array; use datafusion_common::utils::{ SingleRowListArrayBuilder, compare_rows, get_row_at_idx, take_function_args, }; -use datafusion_common::{Result, ScalarValue, assert_eq_or_internal_err, exec_err}; +use datafusion_common::{ + Result, ScalarValue, assert_eq_or_internal_err, exec_err, internal_err, +}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; use datafusion_expr::utils::format_state_name; use datafusion_expr::{ @@ -598,6 +600,9 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { let emit_groups = match emit_to { EmitTo::All => self.num_groups, EmitTo::First(n) => n, + EmitTo::NextBlock => { + return internal_err!("array_agg does not support blocked groups"); + } }; // Step 1: Count entries per group. For EmitTo::First(n), only groups @@ -657,6 +662,9 @@ impl GroupsAccumulator for ArrayAggGroupsAccumulator { match emit_to { EmitTo::All => self.clear_state(), EmitTo::First(_) => self.compact_retained_state(emit_groups)?, + EmitTo::NextBlock => { + return internal_err!("array_agg does not support blocked groups"); + } } let offsets = OffsetBuffer::new(ScalarBuffer::from(offsets)); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index bcccea381324e..90c62fa906735 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -43,7 +43,8 @@ use datafusion_expr::{ use datafusion_functions_aggregate_common::aggregate::avg_distinct::{ DecimalDistinctAvgAccumulator, Float64DistinctAvgAccumulator, }; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; + +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::FlatNullState; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::{ filtered_null_mask, set_nulls, }; @@ -764,7 +765,7 @@ where sums: Vec, /// Track nulls in the input / filters - null_state: NullState, + null_state: FlatNullState, /// Function that computes the final average (value / count) avg_fn: F, @@ -786,7 +787,7 @@ where sum_data_type: sum_data_type.clone(), counts: vec![], sums: vec![], - null_state: NullState::new(), + null_state: FlatNullState::new(None), avg_fn, } } @@ -810,12 +811,17 @@ where // increment counts, update sums self.counts.resize(total_num_groups, 0); self.sums.resize(total_num_groups, T::default_value()); + + // `block_id` is ignored in `value_fn`, because `AvgGroupsAccumulator` + // still not support blocked groups. + // More details can see `GroupsAccumulator::supports_blocked_groups`. self.null_state.accumulate( group_indices, values, opt_filter, total_num_groups, - |group_index, new_value| { + |_, group_index, new_value| { + let group_index = group_index as usize; // SAFETY: group_index is guaranteed to be in bounds let sum = unsafe { self.sums.get_unchecked_mut(group_index) }; *sum = sum.add_wrapping(new_value); @@ -897,28 +903,36 @@ where let partial_sums = values[1].as_primitive::(); // update counts with partial counts self.counts.resize(total_num_groups, 0); + + // `block_id` is ignored in `value_fn`, because `AvgGroupsAccumulator` + // still not support blocked groups. + // More details can see `GroupsAccumulator::supports_blocked_groups`. self.null_state.accumulate( group_indices, partial_counts, opt_filter, total_num_groups, - |group_index, partial_count| { + |_, group_index, partial_count| { // SAFETY: group_index is guaranteed to be in bounds - let count = unsafe { self.counts.get_unchecked_mut(group_index) }; + let count = + unsafe { self.counts.get_unchecked_mut(group_index as usize) }; *count += partial_count; }, ); // update sums self.sums.resize(total_num_groups, T::default_value()); + // `block_id` is ignored in `value_fn`, because `AvgGroupsAccumulator` + // still not support blocked groups. + // More details can see `GroupsAccumulator::supports_blocked_groups`. self.null_state.accumulate( group_indices, partial_sums, opt_filter, total_num_groups, - |group_index, new_value: ::Native| { + |_, group_index, new_value: ::Native| { // SAFETY: group_index is guaranteed to be in bounds - let sum = unsafe { self.sums.get_unchecked_mut(group_index) }; + let sum = unsafe { self.sums.get_unchecked_mut(group_index as usize) }; *sum = sum.add_wrapping(new_value); }, ); diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 8e6bf9205b2fc..71786a0f0605f 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -437,18 +437,21 @@ impl FirstLastGroupsAccumulator { }) } - fn take_orderings(&mut self, emit_to: EmitTo) -> Vec> { + fn take_orderings(&mut self, emit_to: EmitTo) -> Result>> { let result = emit_to.take_needed(&mut self.orderings); match emit_to { EmitTo::All => self.size_of_orderings = 0, EmitTo::First(_) => { self.size_of_orderings -= - result.iter().map(ScalarValue::size_of_vec).sum::() + result.iter().map(ScalarValue::size_of_vec).sum::(); + } + EmitTo::NextBlock => { + return internal_err!("first_last does not support blocked groups"); } } - result + Ok(result) } fn resize_states(&mut self, new_size: usize) { @@ -505,7 +508,7 @@ impl FirstLastGroupsAccumulator { Ok(( self.state.take(emit_to)?, - self.take_orderings(emit_to), + self.take_orderings(emit_to)?, state::take_need(&mut self.is_sets, emit_to), )) } diff --git a/datafusion/functions-aggregate/src/first_last/state.rs b/datafusion/functions-aggregate/src/first_last/state.rs index 3aa578c74c038..83b31af5bc20e 100644 --- a/datafusion/functions-aggregate/src/first_last/state.rs +++ b/datafusion/functions-aggregate/src/first_last/state.rs @@ -309,6 +309,9 @@ pub(crate) fn take_need( } first_n } + EmitTo::NextBlock => { + unreachable!("first_last does not support blocked groups") + } } } diff --git a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs index e4ac7eccf5692..a94de4c94248f 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -200,7 +200,7 @@ impl GroupsAccumulator for MinMaxBytesAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let (data_capacity, min_maxes) = self.inner.emit_to(emit_to); + let (data_capacity, min_maxes) = self.inner.emit_to(emit_to)?; // Convert the Vec of bytes to a vec of Strings (at no cost) fn bytes_to_str( @@ -484,13 +484,13 @@ impl MinMaxBytesState { /// /// - `data_capacity`: the total length of all strings and their contents, /// - `min_maxes`: the actual min/max values for each group - fn emit_to(&mut self, emit_to: EmitTo) -> (usize, Vec>>) { + fn emit_to(&mut self, emit_to: EmitTo) -> Result<(usize, Vec>>)> { match emit_to { EmitTo::All => { - ( + Ok(( std::mem::take(&mut self.total_data_bytes), // reset total bytes and min_max std::mem::take(&mut self.min_max), - ) + )) } EmitTo::First(n) => { let first_min_maxes: Vec<_> = self.min_max.drain(..n).collect(); @@ -499,7 +499,10 @@ impl MinMaxBytesState { .map(|opt| opt.as_ref().map(|s| s.len()).unwrap_or(0)) .sum(); self.total_data_bytes -= first_data_capacity; - (first_data_capacity, first_min_maxes) + Ok((first_data_capacity, first_min_maxes)) + } + EmitTo::NextBlock => { + internal_err!("min/max bytes does not support blocked groups") } } } diff --git a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs index 796fd586ca5c8..99023cd8581e7 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_struct.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_struct.rs @@ -99,7 +99,7 @@ impl GroupsAccumulator for MinMaxStructAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let (_, min_maxes) = self.inner.emit_to(emit_to); + let (_, min_maxes) = self.inner.emit_to(emit_to)?; let fields = match &self.inner.data_type { DataType::Struct(fields) => fields, _ => return internal_err!("Data type is not a struct"), @@ -273,13 +273,13 @@ impl MinMaxStructState { /// /// - `data_capacity`: the total length of all strings and their contents, /// - `min_maxes`: the actual min/max values for each group - fn emit_to(&mut self, emit_to: EmitTo) -> (usize, Vec>) { + fn emit_to(&mut self, emit_to: EmitTo) -> Result<(usize, Vec>)> { match emit_to { EmitTo::All => { - ( + Ok(( std::mem::take(&mut self.total_data_bytes), // reset total bytes and min_max std::mem::take(&mut self.min_max), - ) + )) } EmitTo::First(n) => { let first_min_maxes: Vec<_> = self.min_max.drain(..n).collect(); @@ -288,7 +288,10 @@ impl MinMaxStructState { .map(|opt| opt.as_ref().map(|s| s.len()).unwrap_or(0)) .sum(); self.total_data_bytes -= first_data_capacity; - (first_data_capacity, first_min_maxes) + Ok((first_data_capacity, first_min_maxes)) + } + EmitTo::NextBlock => { + internal_err!("min/max struct does not support blocked groups") } } } diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 3fd2b42b2e4af..7441fc0660276 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -18,10 +18,10 @@ pub(crate) mod groups_accumulator { #[expect(unused_imports)] pub(crate) mod accumulate { - pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; + pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::FlatNullState; } pub use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - GroupsAccumulatorAdapter, accumulate::NullState, + GroupsAccumulatorAdapter, accumulate::FlatNullState, }; } pub(crate) mod stats { diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index bedd348dab92f..999a003fa3c24 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -51,7 +51,7 @@ pub mod execution_props { pub use datafusion_expr::var_provider::{VarProvider, VarType}; } -pub use aggregate::groups_accumulator::{GroupsAccumulatorAdapter, NullState}; +pub use aggregate::groups_accumulator::{FlatNullState, GroupsAccumulatorAdapter}; pub use analysis::{AnalysisContext, ExprBoundaries, analyze}; pub use equivalence::{ AcrossPartitions, ConstExpr, EquivalenceProperties, calculate_union, diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 2f3b1a19e7d73..afcfbf4306eaf 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -24,7 +24,7 @@ use arrow::array::types::{ }; use arrow::array::{ArrayRef, downcast_primitive}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; use datafusion_expr::EmitTo; @@ -113,6 +113,51 @@ pub trait GroupValues: Send { /// Clear the contents and shrink the capacity to the size of the batch (free up memory usage) fn clear_shrink(&mut self, num_rows: usize); + + /// Returns `true` if this accumulator supports blocked groups. + /// + /// Blocked groups(or called blocked management approach) is an optimization + /// to reduce the cost of managing aggregation intermediate states. + /// + /// Here is brief introduction for two states management approaches: + /// - Blocked approach, states are stored and managed in multiple `Vec`s, + /// we call it `Block`s. Organize like this is for avoiding to resize `Vec` + /// and allocate a new `Vec` instead to reduce cost and get better performance. + /// When locating data in `Block`s, we need to use `block_id` to locate the + /// needed `Block` at first, and use `block_offset` to locate the needed + /// data in `Block` after. + /// + /// - Single approach, all states are stored and managed in a single large `Block`. + /// So when locating data, `block_id` will always be 0, and we only need `block_offset` + /// to locate data in the single `Block`. + /// + /// More details can see: + /// + /// + fn supports_blocked_groups(&self) -> bool { + false + } + + /// Alter the block size in the `group values` + /// + /// If the target block size is `None`, it will use a single big + /// block(can think it a `Vec`) to manage the state. + /// + /// If the target block size` is `Some(blk_size)`, it will try to + /// set the block size to `blk_size`, and the try will only success + /// when the `group values` has supported blocked mode. + /// + /// NOTICE: After altering block size, all data in previous will be cleared. + /// + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { + if block_size.is_some() { + return Err(DataFusionError::NotImplemented( + "this group values doesn't support blocked mode yet".to_string(), + )); + } + + Ok(()) + } } /// Return a specialized implementation of [`GroupValues`] for the given schema. diff --git a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs index cc4576eabddbd..4316665dec090 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/multi_group_by/mod.rs @@ -41,7 +41,7 @@ use arrow::datatypes::{ }; use datafusion_common::hash_utils::RandomState; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::{Result, internal_datafusion_err, not_impl_err}; +use datafusion_common::{Result, internal_datafusion_err, internal_err, not_impl_err}; use datafusion_execution::memory_pool::proxy::{HashTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; @@ -1161,6 +1161,11 @@ impl GroupValues for GroupValuesColumn { output } + EmitTo::NextBlock => { + return internal_err!( + "group_values_column does not support blocked groups" + ); + } }; // TODO: Materialize dictionaries in group keys (#7647) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index a3bd31f76c233..a34c8465207fd 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -23,9 +23,8 @@ use arrow::array::{ use arrow::compute::cast; use arrow::datatypes::{DataType, SchemaRef}; use arrow::row::{RowConverter, Rows, SortField}; -use datafusion_common::Result; -use datafusion_common::hash_utils::RandomState; -use datafusion_common::hash_utils::create_hashes; +use datafusion_common::hash_utils::{RandomState, create_hashes}; +use datafusion_common::{Result, internal_err}; use datafusion_execution::memory_pool::proxy::{HashTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use hashbrown::hash_table::HashTable; @@ -233,6 +232,11 @@ impl GroupValues for GroupValuesRows { }); output } + EmitTo::NextBlock => { + return internal_err!( + "group_values_rows does not support blocked groups" + ); + } }; // TODO: Materialize dictionaries in group keys diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs index e993c0c53d199..1a64d45ee99bf 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/boolean.rs @@ -20,7 +20,7 @@ use crate::aggregates::group_values::GroupValues; use arrow::array::{ ArrayRef, AsArray as _, BooleanArray, BooleanBufferBuilder, NullBufferBuilder, }; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_expr::EmitTo; use std::{mem::size_of, sync::Arc}; @@ -103,6 +103,11 @@ impl GroupValues for GroupValuesBoolean { let emit_count = match emit_to { EmitTo::All => len, EmitTo::First(n) => n, + EmitTo::NextBlock => { + return internal_err!( + "group_values_boolean does not support blocked groups" + ); + } }; builder.append_n(emit_count, false); if let Some(idx) = self.true_group.as_mut() { diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs index b881a51b25474..9aeccb3f38b67 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes.rs @@ -20,7 +20,7 @@ use std::mem::size_of; use crate::aggregates::group_values::GroupValues; use arrow::array::{Array, ArrayRef, OffsetSizeTrait}; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_expr::EmitTo; use datafusion_physical_expr_common::binary_map::{ArrowBytesMap, OutputType}; @@ -115,6 +115,11 @@ impl GroupValues for GroupValuesBytes { emit_group_values } + EmitTo::NextBlock => { + return internal_err!( + "group_values_bytes does not support blocked groups" + ); + } }; Ok(vec![group_values]) diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs index 7a56f7c52c11a..60cf2c4a578b6 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/bytes_view.rs @@ -17,6 +17,7 @@ use crate::aggregates::group_values::GroupValues; use arrow::array::{Array, ArrayRef}; +use datafusion_common::internal_err; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewMap; @@ -117,6 +118,11 @@ impl GroupValues for GroupValuesBytesView { emit_group_values } + EmitTo::NextBlock => { + return internal_err!( + "group_values_bytes_view does not support blocked groups" + ); + } }; Ok(vec![group_values]) diff --git a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs index efaf7eba0f1b5..9362fce72b455 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/single_group_by/primitive.rs @@ -22,12 +22,16 @@ use arrow::array::{ cast::AsArray, }; use arrow::datatypes::{DataType, i256}; -use datafusion_common::Result; +use datafusion_common::{Result, internal_err}; use datafusion_common::hash_utils::RandomState; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::group_index_operations::{ + BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, +}; use half::f16; use hashbrown::hash_table::HashTable; +use std::{collections::VecDeque, mem}; #[cfg(not(feature = "force_hash_collisions"))] use std::hash::BuildHasher; use std::mem::size_of; @@ -88,24 +92,43 @@ pub struct GroupValuesPrimitive { /// is obvious in high cardinality group by situation. /// More details can see: /// - map: HashTable<(usize, u64)>, + map: HashTable<(u64, u64)>, /// The group index of the null value if any - null_group: Option, + null_group: Option, + /// The values for each group index - values: Vec, + values: VecDeque>, + /// The random state used to generate hashes random_state: RandomState, + + /// Block size of current `GroupValues` if exist: + /// - If `None`, it means block optimization is disabled, + /// all `group values`` will be stored in a single `Vec` + /// + /// - If `Some(blk_size)`, it means block optimization is enabled, + /// `group values` will be stored in multiple `Vec`s, and each + /// `Vec` if of `blk_size` len, and we call it a `block` + /// + block_size: Option, } impl GroupValuesPrimitive { pub fn new(data_type: DataType) -> Self { assert!(PrimitiveArray::::is_compatible(&data_type)); + + // As a optimization, we ensure the `single block` always exist + // in flat mode, it can eliminate an expansive row-level empty checking + let mut values = VecDeque::new(); + values.push_back(Vec::new()); + Self { data_type, map: HashTable::with_capacity(128), - values: Vec::with_capacity(128), + values, null_group: None, random_state: crate::aggregates::AGGREGATION_HASH_SEED, + block_size: None, } } } @@ -115,56 +138,172 @@ where T::Native: HashValue, { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { + if let Some(block_size) = self.block_size { + let before_add_group = |group_values: &mut VecDeque>| { + if group_values.is_empty() + || group_values.back().unwrap().len() == block_size + { + let new_block = Vec::with_capacity(block_size); + group_values.push_back(new_block); + } + }; + self.get_or_create_groups_internal::<_, BlockedGroupIndexOperations>( + cols, + groups, + before_add_group, + ) + } else { + self.get_or_create_groups_internal::<_, FlatGroupIndexOperations>( + cols, + groups, + |_: &mut VecDeque>| {}, + ) + } + } + + fn size(&self) -> usize { + self.map.capacity() * size_of::<(usize, u64)>() + + self + .values + .iter() + .map(|blk| blk.len() * blk.allocated_size()) + .sum::() + } + + fn is_empty(&self) -> bool { + self.len() == 0 + } + + fn len(&self) -> usize { + self.values.iter().map(|block| block.len()).sum::() + } + + fn emit(&mut self, emit_to: EmitTo) -> Result> { + if self.block_size.is_some() { + self.emit_groups_internal::(emit_to) + } else { + self.emit_groups_internal::(emit_to) + } + } + + fn clear_shrink(&mut self, num_rows: usize) { + // TODO: Only reserve room of values in `flat mode` currently, + // we may need to consider it again when supporting spilling + // for `blocked mode`. + if self.block_size.is_none() { + let single_block = self.values.back_mut().unwrap(); + single_block.clear(); + single_block.shrink_to(num_rows); + } + + self.map.clear(); + self.map.shrink_to(num_rows, |_| 0); // hasher does not matter since the map is cleared + } + + fn supports_blocked_groups(&self) -> bool { + true + } + + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { + self.map.clear(); + self.values.clear(); + self.null_group = None; + self.block_size = block_size; + + // As mentioned above, we ensure the `single block` always exist + // in `flat mode` + if block_size.is_none() { + self.values.push_back(Vec::new()); + } + + Ok(()) + } +} + +impl GroupValuesPrimitive +where + T::Native: HashValue, +{ + fn get_or_create_groups_internal( + &mut self, + cols: &[ArrayRef], + groups: &mut Vec, + mut before_add_group: F, + ) -> Result<()> + where + F: FnMut(&mut VecDeque>), + O: GroupIndexOperations, + { assert_eq!(cols.len(), 1); groups.clear(); for v in cols[0].as_primitive::() { - let group_id = match v { + let packed_index = match v { None => *self.null_group.get_or_insert_with(|| { - let group_id = self.values.len(); - self.values.push(Default::default()); - group_id + // Actions before add new group like checking if room is enough + before_add_group(&mut self.values); + + // Get block infos and update block, + // we need `current block` and `next offset in block` + let block_id = self.values.len() as u32 - 1; + // Safety: we have already ensured that the block exists + let current_block = self.values.back_mut().unwrap(); + let block_offset = current_block.len() as u64; + current_block.push(Default::default()); + + // Get group index and finish actions needed it + O::pack_index(block_id, block_offset) }), Some(key) => { let state = &self.random_state; let hash = key.hash(state); let insert = self.map.entry( hash, - |&(g, h)| unsafe { - hash == h && self.values.get_unchecked(g).is_eq(key) + |&(idx, _)| unsafe { + let block_id = O::get_block_id(idx); + let block_offset = O::get_block_offset(idx); + self.values + .get(block_id as usize) + .unwrap() + .get_unchecked(block_offset as usize) + .is_eq(key) }, |&(_, h)| h, ); match insert { - hashbrown::hash_table::Entry::Occupied(o) => o.get().0, + hashbrown::hash_table::Entry::Occupied(o) => { + let (idx, _) = *o.get(); + idx + } hashbrown::hash_table::Entry::Vacant(v) => { - let g = self.values.len(); - v.insert((g, hash)); - self.values.push(key); - g + // Actions before add new group like checking if room is enough + before_add_group(&mut self.values); + + // Get block infos and update block, + // we need `current block` and `next offset in block` + let block_id = self.values.len() as u32 - 1; + let current_block = self.values.back_mut().unwrap(); + let block_offset = current_block.len() as u64; + current_block.push(key); + + // Get group index and finish actions needed it + let packed_index = O::pack_index(block_id, block_offset); + v.insert((packed_index, hash)); + packed_index } } } }; - groups.push(group_id) + groups.push(packed_index as usize) } Ok(()) } - fn size(&self) -> usize { - self.map.capacity() * size_of::<(usize, u64)>() + self.values.allocated_size() - } - - fn is_empty(&self) -> bool { - self.values.is_empty() - } - - fn len(&self) -> usize { - self.values.len() - } - - fn emit(&mut self, emit_to: EmitTo) -> Result> { + fn emit_groups_internal( + &mut self, + emit_to: EmitTo, + ) -> Result> { fn build_primitive( values: Vec, null_idx: Option, @@ -180,46 +319,264 @@ where PrimitiveArray::::new(values.into(), nulls) } - let array: PrimitiveArray = match emit_to { - EmitTo::All => { + let array: PrimitiveArray = match (self.block_size, emit_to) { + // =============================================== + // Emitting in flat mode + // =============================================== + (None, EmitTo::All) => { self.map.clear(); - build_primitive(std::mem::take(&mut self.values), self.null_group.take()) + let values = mem::take(self.values.back_mut().unwrap()); + let null_group_opt = self.null_group.take().map(|packed_index| { + let blk_offset = O::get_block_offset(packed_index); + blk_offset as usize + }); + build_primitive(values, null_group_opt) } - EmitTo::First(n) => { + (None, EmitTo::First(n)) => { + let n = n as u64; + // Decrement group index by n in `map` self.map.retain(|entry| { - // Decrement group index by n - let group_idx = entry.0; - match group_idx.checked_sub(n) { + let packed_index = entry.0; + let blk_offset = O::get_block_offset(packed_index); + match blk_offset.checked_sub(n) { // Group index was >= n, shift value down Some(sub) => { - entry.0 = sub; + let packed_index = O::pack_index(0, sub); + entry.0 = packed_index; true } // Group index was < n, so remove from table None => false, } }); - let null_group = match &mut self.null_group { + + // Decrement group index by n in `null_group` + let null_group_opt = match &mut self.null_group { Some(v) if *v >= n => { - *v -= n; + let mut blk_offset = O::get_block_offset(*v); + blk_offset -= n; + *v = O::pack_index(0, blk_offset); None } - Some(_) => self.null_group.take(), + Some(_) => self.null_group.take().map(|packed_index| { + let blk_offset = O::get_block_offset(packed_index); + blk_offset as usize + }), None => None, }; - let mut split = self.values.split_off(n); - std::mem::swap(&mut self.values, &mut split); - build_primitive(split, null_group) + + let single_block = self.values.back_mut().unwrap(); + let mut split = single_block.split_off(n as usize); + mem::swap(single_block, &mut split); + build_primitive(split, null_group_opt) + } + // =============================================== + // Emitting in blocked mode + // =============================================== + (Some(_), EmitTo::NextBlock) => { + // Similar as `EmitTo:All`, we will clear the old index infos both + // in `map` and `null_group` + self.map.clear(); + + let null_block_pair_opt = self.null_group.map(|packed_index| { + ( + O::get_block_id(packed_index), + O::get_block_offset(packed_index), + ) + }); + let null_idx = match null_block_pair_opt { + Some((blk_id, blk_offset)) if blk_id > 0 => { + let new_blk_id = blk_id - 1; + let new_packed_index = O::pack_index(new_blk_id, blk_offset); + self.null_group = Some(new_packed_index); + None + } + Some((_, blk_offset)) => { + self.null_group = None; + Some(blk_offset as usize) + } + None => None, + }; + + let emit_blk = self.values.pop_front().unwrap(); + build_primitive(emit_blk, null_idx) + } + (blk_size, emit_to) => { + return internal_err!( + "invalid emit_to for mode, block_size:{blk_size:?}, emit_to:{emit_to:?}" + ); } }; Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) } +} - fn clear_shrink(&mut self, num_rows: usize) { - self.values.clear(); - self.values.shrink_to(num_rows); - self.map.clear(); - self.map.shrink_to(num_rows, |_| 0); // hasher does not matter since the map is cleared +#[cfg(test)] +mod tests { + use std::collections::BTreeMap; + use std::sync::Arc; + + use crate::aggregates::group_values::single_group_by::primitive::GroupValuesPrimitive; + use crate::aggregates::group_values::GroupValues; + use arrow::array::{AsArray, UInt32Array}; + use arrow::datatypes::{DataType, UInt32Type}; + use datafusion_expr::EmitTo; + use datafusion_functions_aggregate_common::aggregate::groups_accumulator::group_index_operations::{ + BlockedGroupIndexOperations, GroupIndexOperations, + }; + + #[test] + fn test_flat_primitive_group_values() { + // Will cover such insert cases: + // 1.1 Non-null row + distinct + // 1.2 Null row + distinct + // 1.3 Non-null row + non-distinct + // 1.4 Null row + non-distinct + // + // Will cover such emit cases: + // 2.1 Emit first n + // 2.2 Emit all + // 2.3 Insert again + emit + let mut group_values = GroupValuesPrimitive::::new(DataType::UInt32); + let mut group_indices = vec![]; + + let data1 = Arc::new(UInt32Array::from(vec![ + Some(1), + None, + Some(1), + None, + Some(2), + Some(3), + ])); + let data2 = Arc::new(UInt32Array::from(vec![Some(3), None, Some(4), Some(5)])); + + // Insert case 1.1, 1.3, 1.4 + Emit case 2.1 + group_values + .intern(&[Arc::clone(&data1) as _], &mut group_indices) + .unwrap(); + + let mut expected = BTreeMap::new(); + for (&group_index, value) in group_indices.iter().zip(data1.iter()) { + expected.insert(group_index, value); + } + let mut expected = expected.into_iter().collect::>(); + let last_group_index = expected.len() - 1; + let last_value = expected.last().unwrap().1; + expected.pop(); + + let emit_result = group_values.emit(EmitTo::First(3)).unwrap(); + let actual = emit_result[0] + .as_primitive::() + .iter() + .enumerate() + .map(|(group_idx, val)| { + assert!(group_idx < last_group_index); + (group_idx, val) + }) + .collect::>(); + + assert_eq!(expected, actual); + + // Insert case 1.1~1.3 + Emit case 2.2~2.3 + group_values + .intern(&[Arc::clone(&data2) as _], &mut group_indices) + .unwrap(); + + let mut expected = BTreeMap::new(); + for (&group_index, value) in group_indices.iter().zip(data2.iter()) { + if group_index == 0 { + assert_eq!(last_value, value); + } + expected.insert(group_index, value); + } + let expected = expected.into_iter().collect::>(); + + let emit_result = group_values.emit(EmitTo::All).unwrap(); + let actual = emit_result[0] + .as_primitive::() + .iter() + .enumerate() + .collect::>(); + + assert_eq!(expected, actual); + } + + #[test] + fn test_blocked_primitive_group_values() { + // Will cover such insert cases: + // 1.1 Non-null row + distinct + // 1.2 Null row + distinct + // 1.3 Non-null row + non-distinct + // 1.4 Null row + non-distinct + // + // Will cover such emit cases: + // 2.1 Emit block + // 2.2 Insert again + emit block + // + let mut group_values = GroupValuesPrimitive::::new(DataType::UInt32); + let block_size = 2; + group_values.alter_block_size(Some(block_size)).unwrap(); + let mut group_indices = vec![]; + + let data1 = Arc::new(UInt32Array::from(vec![ + Some(1), + None, + Some(1), + None, + Some(2), + Some(3), + ])); + let data2 = Arc::new(UInt32Array::from(vec![Some(3), None, Some(4)])); + + // Insert case 1.1, 1.3, 1.4 + Emit case 2.1 + group_values + .intern(&[Arc::clone(&data1) as _], &mut group_indices) + .unwrap(); + + let mut expected = BTreeMap::new(); + for (&packed_index, value) in group_indices.iter().zip(data1.iter()) { + let block_id = BlockedGroupIndexOperations::get_block_id(packed_index as u64); + let block_offset = + BlockedGroupIndexOperations::get_block_offset(packed_index as u64); + let flatten_index = block_id as usize * block_size + block_offset as usize; + expected.insert(flatten_index, value); + } + let expected = expected.into_iter().collect::>(); + + let emit_result1 = group_values.emit(EmitTo::NextBlock).unwrap(); + assert_eq!(emit_result1[0].len(), block_size); + let emit_result2 = group_values.emit(EmitTo::NextBlock).unwrap(); + assert_eq!(emit_result2[0].len(), block_size); + let iter1 = emit_result1[0].as_primitive::().iter(); + let iter2 = emit_result2[0].as_primitive::().iter(); + let actual = iter1.chain(iter2).enumerate().collect::>(); + + assert_eq!(actual, expected); + + // Insert case 1.1~1.2 + Emit case 2.2 + group_values + .intern(&[Arc::clone(&data2) as _], &mut group_indices) + .unwrap(); + + let mut expected = BTreeMap::new(); + for (&packed_index, value) in group_indices.iter().zip(data2.iter()) { + let block_id = BlockedGroupIndexOperations::get_block_id(packed_index as u64); + let block_offset = + BlockedGroupIndexOperations::get_block_offset(packed_index as u64); + let flatten_index = block_id as usize * block_size + block_offset as usize; + expected.insert(flatten_index, value); + } + let expected = expected.into_iter().collect::>(); + + let emit_result1 = group_values.emit(EmitTo::NextBlock).unwrap(); + assert_eq!(emit_result1[0].len(), block_size); + let emit_result2 = group_values.emit(EmitTo::NextBlock).unwrap(); + assert_eq!(emit_result2[0].len(), 1); + let iter1 = emit_result1[0].as_primitive::().iter(); + let iter2 = emit_result2[0].as_primitive::().iter(); + let actual = iter1.chain(iter2).enumerate().collect::>(); + + assert_eq!(actual, expected); } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 79a21d1c345ec..a882872a87963 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2213,6 +2213,10 @@ mod tests { use futures::{FutureExt, Stream}; use insta::{allow_duplicates, assert_snapshot}; + // ========================================================================= + // Helper functions & test fixtures + // ========================================================================= + // Generate a schema which consists of 5 columns (a, b, c, d, e) fn create_test_schema() -> Result { let a = Field::new("a", DataType::Int32, true); @@ -2308,8 +2312,16 @@ mod tests { ) } - fn new_spill_ctx(batch_size: usize, max_memory: usize) -> Arc { - let session_config = SessionConfig::new().with_batch_size(batch_size); + fn new_spill_ctx( + batch_size: usize, + max_memory: usize, + enable_blocked_groups: bool, + ) -> Arc { + let mut session_config = SessionConfig::new().with_batch_size(batch_size); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked_groups)), + ); let runtime = RuntimeEnvBuilder::new() .with_memory_pool(Arc::new(FairSpillPool::new(max_memory))) .build_arc() @@ -2320,9 +2332,19 @@ mod tests { Arc::new(task_ctx) } + /// Creates a TaskContext with the blocked groups configuration for testing. + fn task_ctx_with_blocked_groups(enable_blocked_groups: bool) -> Arc { + let session_config = SessionConfig::new().set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked_groups)), + ); + Arc::new(TaskContext::default().with_session_config(session_config)) + } + async fn check_grouping_sets( input: Arc, spill: bool, + enable_blocked_groups: bool, ) -> Result<()> { let input_schema = input.schema(); @@ -2352,9 +2374,9 @@ mod tests { let task_ctx = if spill { // adjust the max memory size to have the partial aggregate result for spill mode. - new_spill_ctx(4, 500) + new_spill_ctx(4, 500, enable_blocked_groups) } else { - Arc::new(TaskContext::default()) + task_ctx_with_blocked_groups(enable_blocked_groups) }; let partial_aggregate = Arc::new(AggregateExec::try_new( @@ -2432,7 +2454,7 @@ mod tests { let final_grouping_set = grouping_set.as_final(); let task_ctx = if spill { - new_spill_ctx(4, 3160) + new_spill_ctx(4, 3160, enable_blocked_groups) } else { task_ctx }; @@ -2483,7 +2505,11 @@ mod tests { } /// build the aggregates on the data from some_data() and check the results - async fn check_aggregates(input: Arc, spill: bool) -> Result<()> { + async fn check_aggregates( + input: Arc, + spill: bool, + enable_blocked_groups: bool, + ) -> Result<()> { let input_schema = input.schema(); let grouping_set = PhysicalGroupBy::new( @@ -2502,9 +2528,9 @@ mod tests { let task_ctx = if spill { // set to an appropriate value to trigger spill - new_spill_ctx(2, 1600) + new_spill_ctx(2, 1600, enable_blocked_groups) } else { - Arc::new(TaskContext::default()) + task_ctx_with_blocked_groups(enable_blocked_groups) }; let partial_aggregate = Arc::new(AggregateExec::try_new( @@ -2566,7 +2592,7 @@ mod tests { let task_ctx = if spill { // enlarge memory limit to let the final aggregation finish - new_spill_ctx(2, 2600) + new_spill_ctx(2, 2600, enable_blocked_groups) } else { Arc::clone(&task_ctx) }; @@ -2759,64 +2785,86 @@ mod tests { } } - //--- Tests ---// + // ========================================================================= + // Basic aggregation correctness tests + // ========================================================================= #[tokio::test] async fn aggregate_source_not_yielding() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(false)); - - check_aggregates(input, false).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(false)); + check_aggregates(input, false, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_grouping_sets_source_not_yielding() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(false)); - - check_grouping_sets(input, false).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(false)); + check_grouping_sets(input, false, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_source_with_yielding() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(true)); - - check_aggregates(input, false).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(true)); + check_aggregates(input, false, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_grouping_sets_with_yielding() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(true)); - - check_grouping_sets(input, false).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(true)); + check_grouping_sets(input, false, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_source_not_yielding_with_spill() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(false)); - - check_aggregates(input, true).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(false)); + check_aggregates(input, true, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_grouping_sets_source_not_yielding_with_spill() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(false)); - - check_grouping_sets(input, true).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(false)); + check_grouping_sets(input, true, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_source_with_yielding_with_spill() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(true)); - - check_aggregates(input, true).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(true)); + check_aggregates(input, true, enable_blocked).await?; + } + Ok(()) } #[tokio::test] async fn aggregate_grouping_sets_with_yielding_with_spill() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(true)); - - check_grouping_sets(input, true).await + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(true)); + check_grouping_sets(input, true, enable_blocked).await?; + } + Ok(()) } + // ========================================================================= + // OOM & cancellation tests + // ========================================================================= + // Median(a) fn test_median_agg_expr(schema: SchemaRef) -> Result { AggregateExprBuilder::new(median_udaf(), vec![col("a", &schema)?]) @@ -2827,74 +2875,83 @@ mod tests { #[tokio::test] async fn test_oom() -> Result<()> { - let input: Arc = Arc::new(TestYieldingExec::new(true)); - let input_schema = input.schema(); - - let runtime = RuntimeEnvBuilder::new() - .with_memory_limit(1, 1.0) - .build_arc()?; - let task_ctx = TaskContext::default().with_runtime(runtime); - let task_ctx = Arc::new(task_ctx); - - let groups_none = PhysicalGroupBy::default(); - let groups_some = PhysicalGroupBy::new( - vec![(col("a", &input_schema)?, "a".to_string())], - vec![], - vec![vec![false]], - false, - ); - - // something that allocates within the aggregator - let aggregates_v0: Vec> = - vec![Arc::new(test_median_agg_expr(Arc::clone(&input_schema))?)]; + for enable_blocked in [false, true] { + let input: Arc = Arc::new(TestYieldingExec::new(true)); + let input_schema = input.schema(); + + let runtime = RuntimeEnvBuilder::new() + .with_memory_limit(1, 1.0) + .build_arc()?; + let mut session_config = SessionConfig::new(); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked)), + ); + let task_ctx = TaskContext::default() + .with_session_config(session_config) + .with_runtime(runtime); + let task_ctx = Arc::new(task_ctx); + + let groups_none = PhysicalGroupBy::default(); + let groups_some = PhysicalGroupBy::new( + vec![(col("a", &input_schema)?, "a".to_string())], + vec![], + vec![vec![false]], + false, + ); - // use fast-path in `row_hash.rs`. - let aggregates_v2: Vec> = vec![Arc::new( - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) - .schema(Arc::clone(&input_schema)) - .alias("AVG(b)") - .build()?, - )]; + // something that allocates within the aggregator + let aggregates_v0: Vec> = + vec![Arc::new(test_median_agg_expr(Arc::clone(&input_schema))?)]; - for (version, groups, aggregates) in [ - (0, groups_none, aggregates_v0), - (2, groups_some, aggregates_v2), - ] { - let n_aggr = aggregates.len(); - let partial_aggregate = Arc::new(AggregateExec::try_new( - AggregateMode::Single, - groups, - aggregates, - vec![None; n_aggr], - Arc::clone(&input), - Arc::clone(&input_schema), - )?); + // use fast-path in `row_hash.rs`. + let aggregates_v2: Vec> = vec![Arc::new( + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("AVG(b)") + .build()?, + )]; + + for (version, groups, aggregates) in [ + (0, groups_none, aggregates_v0), + (2, groups_some, aggregates_v2), + ] { + let n_aggr = aggregates.len(); + let partial_aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + groups, + aggregates, + vec![None; n_aggr], + Arc::clone(&input), + Arc::clone(&input_schema), + )?); - let stream = partial_aggregate.execute_typed(0, &task_ctx)?; + let stream = partial_aggregate.execute_typed(0, &task_ctx)?; - // ensure that we really got the version we wanted - match version { - 0 => { - assert!(matches!(stream, StreamType::AggregateStream(_))); - } - 1 => { - assert!(matches!(stream, StreamType::GroupedHash(_))); - } - 2 => { - assert!(matches!(stream, StreamType::GroupedHash(_))); + // ensure that we really got the version we wanted + match version { + 0 => { + assert!(matches!(stream, StreamType::AggregateStream(_))); + } + 1 => { + assert!(matches!(stream, StreamType::GroupedHash(_))); + } + 2 => { + assert!(matches!(stream, StreamType::GroupedHash(_))); + } + _ => panic!("Unknown version: {version}"), } - _ => panic!("Unknown version: {version}"), - } - let stream: SendableRecordBatchStream = stream.into(); - let err = collect(stream).await.unwrap_err(); + let stream: SendableRecordBatchStream = stream.into(); + let err = collect(stream).await.unwrap_err(); - // error root cause traversal is a bit complicated, see #4172. - let err = err.find_root(); - assert!( - matches!(err, DataFusionError::ResourcesExhausted(_)), - "Wrong error type: {err}", - ); + // error root cause traversal is a bit complicated, see #4172. + let err = err.find_root(); + assert!( + matches!(err, DataFusionError::ResourcesExhausted(_)), + "Wrong error type: {err}", + ); + } } Ok(()) @@ -2938,48 +2995,57 @@ mod tests { #[tokio::test] async fn test_drop_cancel_with_groups() -> Result<()> { - let task_ctx = Arc::new(TaskContext::default()); - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Float64, true), - Field::new("b", DataType::Float64, true), - ])); + for enable_blocked in [false, true] { + let task_ctx = task_ctx_with_blocked_groups(enable_blocked); + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Float64, true), + Field::new("b", DataType::Float64, true), + ])); - let groups = - PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); + let groups = + PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); - let aggregates: Vec> = vec![Arc::new( - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("AVG(b)") - .build()?, - )]; + let aggregates: Vec> = vec![Arc::new( + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(b)") + .build()?, + )]; - let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); - let refs = blocking_exec.refs(); - let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - groups, - aggregates.clone(), - vec![None], - blocking_exec, - schema, - )?); + let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 1)); + let refs = blocking_exec.refs(); + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + groups, + aggregates.clone(), + vec![None], + blocking_exec, + schema, + )?); - let fut = crate::collect(aggregate_exec, task_ctx); - let mut fut = fut.boxed(); + let fut = crate::collect(aggregate_exec, task_ctx); + let mut fut = fut.boxed(); - assert_is_pending(&mut fut); - drop(fut); - assert_strong_count_converges_to_zero(refs).await; + assert_is_pending(&mut fut); + drop(fut); + assert_strong_count_converges_to_zero(refs).await; + } Ok(()) } + // ========================================================================= + // Ordered aggregation tests (FIRST_VALUE / LAST_VALUE) + // ========================================================================= + #[tokio::test] async fn run_first_last_multi_partitions() -> Result<()> { for is_first_acc in [false, true] { for spill in [false, true] { - first_last_multi_partitions(is_first_acc, spill, 4200).await? + for enable_blocked in [false, true] { + first_last_multi_partitions(is_first_acc, spill, 4200, enable_blocked) + .await? + } } } Ok(()) @@ -3035,11 +3101,12 @@ mod tests { is_first_acc: bool, spill: bool, max_memory: usize, + enable_blocked_groups: bool, ) -> Result<()> { let task_ctx = if spill { - new_spill_ctx(2, max_memory) + new_spill_ctx(2, max_memory, enable_blocked_groups) } else { - Arc::new(TaskContext::default()) + task_ctx_with_blocked_groups(enable_blocked_groups) }; let (schema, data) = some_data_v2(); @@ -3119,6 +3186,10 @@ mod tests { Ok(()) } + // ========================================================================= + // Schema & planning tests + // ========================================================================= + #[tokio::test] async fn test_get_finest_requirements() -> Result<()> { let test_schema = create_test_schema()?; @@ -3229,79 +3300,86 @@ mod tests { #[tokio::test] async fn test_agg_exec_group_by_const() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Float32, true), - Field::new("b", DataType::Float32, true), - Field::new("const", DataType::Int32, false), - ])); - - let col_a = col("a", &schema)?; - let col_b = col("b", &schema)?; - let const_expr = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))); - - let groups = PhysicalGroupBy::new( - vec![ - (col_a, "a".to_string()), - (col_b, "b".to_string()), - (const_expr, "const".to_string()), - ], - vec![ - ( - Arc::new(Literal::new(ScalarValue::Float32(None))), - "a".to_string(), - ), - ( - Arc::new(Literal::new(ScalarValue::Float32(None))), - "b".to_string(), - ), - ( - Arc::new(Literal::new(ScalarValue::Int32(None))), - "const".to_string(), - ), - ], - vec![ - vec![false, true, true], - vec![true, false, true], - vec![true, true, false], - ], - true, - ); + for enable_blocked in [false, true] { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Float32, true), + Field::new("b", DataType::Float32, true), + Field::new("const", DataType::Int32, false), + ])); + + let col_a = col("a", &schema)?; + let col_b = col("b", &schema)?; + let const_expr = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))); + + let groups = PhysicalGroupBy::new( + vec![ + (col_a, "a".to_string()), + (col_b, "b".to_string()), + (const_expr, "const".to_string()), + ], + vec![ + ( + Arc::new(Literal::new(ScalarValue::Float32(None))), + "a".to_string(), + ), + ( + Arc::new(Literal::new(ScalarValue::Float32(None))), + "b".to_string(), + ), + ( + Arc::new(Literal::new(ScalarValue::Int32(None))), + "const".to_string(), + ), + ], + vec![ + vec![false, true, true], + vec![true, false, true], + vec![true, true, false], + ], + true, + ); - let aggregates: Vec> = vec![ - AggregateExprBuilder::new(count_udaf(), vec![lit(1)]) - .schema(Arc::clone(&schema)) - .alias("1") - .build() - .map(Arc::new)?, - ]; + let aggregates: Vec> = vec![ + AggregateExprBuilder::new(count_udaf(), vec![lit(1)]) + .schema(Arc::clone(&schema)) + .alias("1") + .build() + .map(Arc::new)?, + ]; - let input_batches = (0..4) - .map(|_| { - let a = Arc::new(Float32Array::from(vec![0.; 8192])); - let b = Arc::new(Float32Array::from(vec![0.; 8192])); - let c = Arc::new(Int32Array::from(vec![1; 8192])); + let input_batches = (0..4) + .map(|_| { + let a = Arc::new(Float32Array::from(vec![0.; 8192])); + let b = Arc::new(Float32Array::from(vec![0.; 8192])); + let c = Arc::new(Int32Array::from(vec![1; 8192])); - RecordBatch::try_new(Arc::clone(&schema), vec![a, b, c]).unwrap() - }) - .collect(); + RecordBatch::try_new(Arc::clone(&schema), vec![a, b, c]).unwrap() + }) + .collect(); - let input = - TestMemoryExec::try_new_exec(&[input_batches], Arc::clone(&schema), None)?; + let input = TestMemoryExec::try_new_exec( + &[input_batches], + Arc::clone(&schema), + None, + )?; - let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::Single, - groups, - aggregates.clone(), - vec![None], - input, - schema, - )?); + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + groups, + aggregates.clone(), + vec![None], + input, + schema, + )?); - let output = - collect(aggregate_exec.execute(0, Arc::new(TaskContext::default()))?).await?; + let output = collect( + aggregate_exec + .execute(0, task_ctx_with_blocked_groups(enable_blocked))?, + ) + .await?; - allow_duplicates! { - assert_snapshot!(batches_to_sort_string(&output), @r" + allow_duplicates! { + assert_snapshot!(batches_to_sort_string(&output), @r" +-----+-----+-------+---------------+-------+ | a | b | const | __grouping_id | 1 | +-----+-----+-------+---------------+-------+ @@ -3310,6 +3388,7 @@ mod tests { | 0.0 | | | 3 | 32768 | +-----+-----+-------+---------------+-------+ "); + } } Ok(()) @@ -3317,106 +3396,113 @@ mod tests { #[tokio::test] async fn test_agg_exec_struct_of_dicts() -> Result<()> { - let batch = RecordBatch::try_new( - Arc::new(Schema::new(vec![ - Field::new( - "labels".to_string(), - DataType::Struct( - vec![ - Field::new( + for enable_blocked in [false, true] { + let batch = RecordBatch::try_new( + Arc::new(Schema::new(vec![ + Field::new( + "labels".to_string(), + DataType::Struct( + vec![ + Field::new( + "a".to_string(), + DataType::Dictionary( + Box::new(DataType::Int32), + Box::new(DataType::Utf8), + ), + true, + ), + Field::new( + "b".to_string(), + DataType::Dictionary( + Box::new(DataType::Int32), + Box::new(DataType::Utf8), + ), + true, + ), + ] + .into(), + ), + false, + ), + Field::new("value", DataType::UInt64, false), + ])), + vec![ + Arc::new(StructArray::from(vec![ + ( + Arc::new(Field::new( "a".to_string(), DataType::Dictionary( Box::new(DataType::Int32), Box::new(DataType::Utf8), ), true, - ), - Field::new( + )), + Arc::new( + vec![Some("a"), None, Some("a")] + .into_iter() + .collect::>(), + ) as ArrayRef, + ), + ( + Arc::new(Field::new( "b".to_string(), DataType::Dictionary( Box::new(DataType::Int32), Box::new(DataType::Utf8), ), true, - ), - ] - .into(), - ), - false, - ), - Field::new("value", DataType::UInt64, false), - ])), - vec![ - Arc::new(StructArray::from(vec![ - ( - Arc::new(Field::new( - "a".to_string(), - DataType::Dictionary( - Box::new(DataType::Int32), - Box::new(DataType::Utf8), - ), - true, - )), - Arc::new( - vec![Some("a"), None, Some("a")] - .into_iter() - .collect::>(), - ) as ArrayRef, - ), - ( - Arc::new(Field::new( - "b".to_string(), - DataType::Dictionary( - Box::new(DataType::Int32), - Box::new(DataType::Utf8), - ), - true, - )), - Arc::new( - vec![Some("b"), Some("c"), Some("b")] - .into_iter() - .collect::>(), - ) as ArrayRef, - ), - ])), - Arc::new(UInt64Array::from(vec![1, 1, 1])), - ], - ) - .expect("Failed to create RecordBatch"); + )), + Arc::new( + vec![Some("b"), Some("c"), Some("b")] + .into_iter() + .collect::>(), + ) as ArrayRef, + ), + ])), + Arc::new(UInt64Array::from(vec![1, 1, 1])), + ], + ) + .expect("Failed to create RecordBatch"); - let group_by = PhysicalGroupBy::new_single(vec![( - col("labels", &batch.schema())?, - "labels".to_string(), - )]); + let group_by = PhysicalGroupBy::new_single(vec![( + col("labels", &batch.schema())?, + "labels".to_string(), + )]); - let aggr_expr = vec![ - AggregateExprBuilder::new(sum_udaf(), vec![col("value", &batch.schema())?]) + let aggr_expr = vec![ + AggregateExprBuilder::new( + sum_udaf(), + vec![col("value", &batch.schema())?], + ) .schema(Arc::clone(&batch.schema())) .alias(String::from("SUM(value)")) .build() .map(Arc::new)?, - ]; + ]; - let input = TestMemoryExec::try_new_exec( - &[vec![batch.clone()]], - Arc::::clone(&batch.schema()), - None, - )?; - let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::FinalPartitioned, - group_by, - aggr_expr, - vec![None], - Arc::clone(&input) as Arc, - batch.schema(), - )?); + let input = TestMemoryExec::try_new_exec( + &[vec![batch.clone()]], + Arc::::clone(&batch.schema()), + None, + )?; + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::FinalPartitioned, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + batch.schema(), + )?); - let session_config = SessionConfig::default(); - let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let session_config = SessionConfig::default().set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked)), + ); + let ctx = TaskContext::default().with_session_config(session_config); + let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; - allow_duplicates! { - assert_snapshot!(batches_to_string(&output), @r" + allow_duplicates! { + assert_snapshot!(batches_to_string(&output), @r" +--------------+------------+ | labels | SUM(value) | +--------------+------------+ @@ -3424,74 +3510,86 @@ mod tests { | {a: , b: c} | 1 | +--------------+------------+ "); + } } Ok(()) } + // ========================================================================= + // Skip aggregation tests + // ========================================================================= + #[tokio::test] async fn test_skip_aggregation_after_first_batch() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("key", DataType::Int32, true), - Field::new("val", DataType::Int32, true), - ])); - - let group_by = - PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); - - let aggr_expr = vec![ - AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) - .schema(Arc::clone(&schema)) - .alias(String::from("COUNT(val)")) - .build() - .map(Arc::new)?, - ]; + for enable_blocked in [false, true] { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, true), + Field::new("val", DataType::Int32, true), + ])); + + let group_by = PhysicalGroupBy::new_single(vec![( + col("key", &schema)?, + "key".to_string(), + )]); + + let aggr_expr = vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("COUNT(val)")) + .build() + .map(Arc::new)?, + ]; - let input_data = vec![ - RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![1, 2, 3])), - Arc::new(Int32Array::from(vec![0, 0, 0])), - ], - ) - .unwrap(), - RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![2, 3, 4])), - Arc::new(Int32Array::from(vec![0, 0, 0])), - ], - ) - .unwrap(), - ]; + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + ]; - let input = - TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; - let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - group_by, - aggr_expr, - vec![None], - Arc::clone(&input) as Arc, - schema, - )?); + let input = + TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + schema, + )?); - let mut session_config = SessionConfig::default(); - session_config = session_config.set( - "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - &ScalarValue::Int64(Some(2)), - ); - session_config = session_config.set( - "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - &ScalarValue::Float64(Some(0.1)), - ); + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &ScalarValue::Int64(Some(2)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &ScalarValue::Float64(Some(0.1)), + ); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked)), + ); - let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let ctx = TaskContext::default().with_session_config(session_config); + let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; - allow_duplicates! { - assert_snapshot!(batches_to_string(&output), @r" + allow_duplicates! { + assert_snapshot!(batches_to_string(&output), @r" +-----+-------------------+ | key | COUNT(val)[count] | +-----+-------------------+ @@ -3503,6 +3601,7 @@ mod tests { | 4 | 1 | +-----+-------------------+ "); + } } Ok(()) @@ -3510,75 +3609,82 @@ mod tests { #[tokio::test] async fn test_skip_aggregation_after_threshold() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("key", DataType::Int32, true), - Field::new("val", DataType::Int32, true), - ])); - - let group_by = - PhysicalGroupBy::new_single(vec![(col("key", &schema)?, "key".to_string())]); - - let aggr_expr = vec![ - AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) - .schema(Arc::clone(&schema)) - .alias(String::from("COUNT(val)")) - .build() - .map(Arc::new)?, - ]; + for enable_blocked in [false, true] { + let schema = Arc::new(Schema::new(vec![ + Field::new("key", DataType::Int32, true), + Field::new("val", DataType::Int32, true), + ])); + + let group_by = PhysicalGroupBy::new_single(vec![( + col("key", &schema)?, + "key".to_string(), + )]); + + let aggr_expr = vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("val", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("COUNT(val)")) + .build() + .map(Arc::new)?, + ]; - let input_data = vec![ - RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![1, 2, 3])), - Arc::new(Int32Array::from(vec![0, 0, 0])), - ], - ) - .unwrap(), - RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![2, 3, 4])), - Arc::new(Int32Array::from(vec![0, 0, 0])), - ], - ) - .unwrap(), - RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![2, 3, 4])), - Arc::new(Int32Array::from(vec![0, 0, 0])), - ], - ) - .unwrap(), - ]; + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![1, 2, 3])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![2, 3, 4])), + Arc::new(Int32Array::from(vec![0, 0, 0])), + ], + ) + .unwrap(), + ]; - let input = - TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; - let aggregate_exec = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - group_by, - aggr_expr, - vec![None], - Arc::clone(&input) as Arc, - schema, - )?); + let input = + TestMemoryExec::try_new_exec(&[input_data], Arc::clone(&schema), None)?; + let aggregate_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr, + vec![None], + Arc::clone(&input) as Arc, + schema, + )?); - let mut session_config = SessionConfig::default(); - session_config = session_config.set( - "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", - &ScalarValue::Int64(Some(5)), - ); - session_config = session_config.set( - "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", - &ScalarValue::Float64(Some(0.1)), - ); + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_rows_threshold", + &ScalarValue::Int64(Some(5)), + ); + session_config = session_config.set( + "datafusion.execution.skip_partial_aggregation_probe_ratio_threshold", + &ScalarValue::Float64(Some(0.1)), + ); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked)), + ); - let ctx = TaskContext::default().with_session_config(session_config); - let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; + let ctx = TaskContext::default().with_session_config(session_config); + let output = collect(aggregate_exec.execute(0, Arc::new(ctx))?).await?; - allow_duplicates! { - assert_snapshot!(batches_to_string(&output), @r" + allow_duplicates! { + assert_snapshot!(batches_to_string(&output), @r" +-----+-------------------+ | key | COUNT(val)[count] | +-----+-------------------+ @@ -3591,6 +3697,7 @@ mod tests { | 4 | 1 | +-----+-------------------+ "); + } } Ok(()) @@ -3642,10 +3749,15 @@ mod tests { Ok(()) } + // ========================================================================= + // Spill & memory limit tests + // ========================================================================= + // test for https://github.com/apache/datafusion/issues/13949 async fn run_test_with_spill_pool_if_necessary( pool_size: usize, expect_spill: bool, + enable_blocked_groups: bool, ) -> Result<()> { fn create_record_batch( schema: &Arc, @@ -3708,10 +3820,15 @@ mod tests { )?); let batch_size = 2; + let mut session_config = SessionConfig::new().with_batch_size(batch_size); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked_groups)), + ); let memory_pool = Arc::new(FairSpillPool::new(pool_size)); let task_ctx = Arc::new( TaskContext::default() - .with_session_config(SessionConfig::new().with_batch_size(batch_size)) + .with_session_config(session_config) .with_runtime(Arc::new( RuntimeEnvBuilder::new() .with_memory_pool(memory_pool) @@ -3769,108 +3886,123 @@ mod tests { #[tokio::test] async fn test_aggregate_with_spill_if_necessary() -> Result<()> { - // test with spill - run_test_with_spill_pool_if_necessary(2_000, true).await?; - // test without spill - run_test_with_spill_pool_if_necessary(20_000, false).await?; + for enable_blocked in [false, true] { + // test with spill + run_test_with_spill_pool_if_necessary(2_000, true, enable_blocked).await?; + // test without spill + run_test_with_spill_pool_if_necessary(20_000, false, enable_blocked).await?; + } Ok(()) } #[tokio::test] async fn test_grouped_aggregation_respects_memory_limit() -> Result<()> { - // test with spill - fn create_record_batch( - schema: &Arc, - data: (Vec, Vec), - ) -> Result { - Ok(RecordBatch::try_new( - Arc::clone(schema), + for enable_blocked in [false, true] { + // test with spill + fn create_record_batch( + schema: &Arc, + data: (Vec, Vec), + ) -> Result { + Ok(RecordBatch::try_new( + Arc::clone(schema), + vec![ + Arc::new(UInt32Array::from(data.0)), + Arc::new(Float64Array::from(data.1)), + ], + )?) + } + + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::UInt32, false), + Field::new("b", DataType::Float64, false), + ])); + + let batches = vec![ + create_record_batch( + &schema, + (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]), + )?, + create_record_batch( + &schema, + (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]), + )?, + ]; + let plan: Arc = + TestMemoryExec::try_new_exec(&[batches], Arc::clone(&schema), None)?; + let proj = ProjectionExec::try_new( vec![ - Arc::new(UInt32Array::from(data.0)), - Arc::new(Float64Array::from(data.1)), + ProjectionExpr::new(lit("0"), "l".to_string()), + ProjectionExpr::new_from_expression(col("a", &schema)?, &schema)?, + ProjectionExpr::new_from_expression(col("b", &schema)?, &schema)?, ], - )?) - } - - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::UInt32, false), - Field::new("b", DataType::Float64, false), - ])); - - let batches = vec![ - create_record_batch(&schema, (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]))?, - create_record_batch(&schema, (vec![2, 3, 4, 4], vec![1.0, 2.0, 3.0, 4.0]))?, - ]; - let plan: Arc = - TestMemoryExec::try_new_exec(&[batches], Arc::clone(&schema), None)?; - let proj = ProjectionExec::try_new( - vec![ - ProjectionExpr::new(lit("0"), "l".to_string()), - ProjectionExpr::new_from_expression(col("a", &schema)?, &schema)?, - ProjectionExpr::new_from_expression(col("b", &schema)?, &schema)?, - ], - plan, - )?; - let plan: Arc = Arc::new(proj); - let schema = plan.schema(); + plan, + )?; + let plan: Arc = Arc::new(proj); + let schema = plan.schema(); - let grouping_set = PhysicalGroupBy::new( - vec![ - (col("l", &schema)?, "l".to_string()), - (col("a", &schema)?, "a".to_string()), - ], - vec![], - vec![vec![false, false]], - false, - ); + let grouping_set = PhysicalGroupBy::new( + vec![ + (col("l", &schema)?, "l".to_string()), + (col("a", &schema)?, "a".to_string()), + ], + vec![], + vec![vec![false, false]], + false, + ); - // Test with MIN for simple intermediate state (min) and AVG for multiple intermediate states (partial sum, partial count). - let aggregates: Vec> = vec![ - Arc::new( - AggregateExprBuilder::new( - datafusion_functions_aggregate::min_max::min_udaf(), - vec![col("b", &schema)?], - ) - .schema(Arc::clone(&schema)) - .alias("MIN(b)") - .build()?, - ), - Arc::new( - AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + // Test with MIN for simple intermediate state (min) and AVG for multiple intermediate states (partial sum, partial count). + let aggregates: Vec> = vec![ + Arc::new( + AggregateExprBuilder::new( + datafusion_functions_aggregate::min_max::min_udaf(), + vec![col("b", &schema)?], + ) .schema(Arc::clone(&schema)) - .alias("AVG(b)") + .alias("MIN(b)") .build()?, - ), - ]; + ), + Arc::new( + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("AVG(b)") + .build()?, + ), + ]; - let single_aggregate = Arc::new(AggregateExec::try_new( - AggregateMode::Single, - grouping_set, - aggregates, - vec![None, None], - plan, - Arc::clone(&schema), - )?); + let single_aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Single, + grouping_set, + aggregates, + vec![None, None], + plan, + Arc::clone(&schema), + )?); - let batch_size = 2; - let memory_pool = Arc::new(FairSpillPool::new(2000)); - let task_ctx = Arc::new( - TaskContext::default() - .with_session_config(SessionConfig::new().with_batch_size(batch_size)) - .with_runtime(Arc::new( - RuntimeEnvBuilder::new() - .with_memory_pool(memory_pool) - .build()?, - )), - ); + let batch_size = 2; + let mut session_config = SessionConfig::new().with_batch_size(batch_size); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_blocked_groups", + &ScalarValue::Boolean(Some(enable_blocked)), + ); + let memory_pool = Arc::new(FairSpillPool::new(2000)); + let task_ctx = Arc::new( + TaskContext::default() + .with_session_config(session_config) + .with_runtime(Arc::new( + RuntimeEnvBuilder::new() + .with_memory_pool(memory_pool) + .build()?, + )), + ); - let result = collect(single_aggregate.execute(0, Arc::clone(&task_ctx))?).await; - match result { - Ok(result) => { - assert_spill_count_metric(true, single_aggregate); + let result = + collect(single_aggregate.execute(0, Arc::clone(&task_ctx))?).await; + match result { + Ok(result) => { + assert_spill_count_metric(true, single_aggregate); - allow_duplicates! { - assert_snapshot!(batches_to_string(&result), @r" + allow_duplicates! { + assert_snapshot!(batches_to_string(&result), @r" +---+---+--------+--------+ | l | a | MIN(b) | AVG(b) | +---+---+--------+--------+ @@ -3879,14 +4011,19 @@ mod tests { | 0 | 4 | 3.0 | 3.5 | +---+---+--------+--------+ "); + } } + Err(e) => assert!(matches!(e, DataFusionError::ResourcesExhausted(_))), } - Err(e) => assert!(matches!(e, DataFusionError::ResourcesExhausted(_))), } Ok(()) } + // ========================================================================= + // Statistics & cardinality estimation tests + // ========================================================================= + #[tokio::test] async fn test_aggregate_statistics_edge_cases() -> Result<()> { use datafusion_common::ColumnStatistics; @@ -4434,6 +4571,16 @@ mod tests { #[tokio::test] async fn test_order_is_retained_when_spilling() -> Result<()> { + for enable_blocked in [false, true] { + check_order_is_retained_when_spilling(enable_blocked, 600).await?; + } + Ok(()) + } + + async fn check_order_is_retained_when_spilling( + enable_blocked_groups: bool, + max_memory: usize, + ) -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, false), Field::new("b", DataType::Int64, false), @@ -4497,7 +4644,7 @@ mod tests { Arc::clone(&schema), )?); - let task_ctx = new_spill_ctx(1, 600); + let task_ctx = new_spill_ctx(1, max_memory, enable_blocked_groups); let result = collect(aggr.execute(0, Arc::clone(&task_ctx))?).await?; assert_spill_count_metric(true, aggr); @@ -4520,6 +4667,16 @@ mod tests { /// ResourcesExhausted rather than silently exceeding memory limits. #[tokio::test] async fn test_sort_reservation_fails_during_spill() -> Result<()> { + for enable_blocked in [false, true] { + check_sort_reservation_fails_during_spill(enable_blocked, 500).await?; + } + Ok(()) + } + + async fn check_sort_reservation_fails_during_spill( + enable_blocked_groups: bool, + max_memory: usize, + ) -> Result<()> { let schema = Arc::new(Schema::new(vec![ Field::new("g", DataType::Int64, false), Field::new("a", DataType::Float64, false), @@ -4627,9 +4784,9 @@ mod tests { Arc::clone(&schema), )?); - // Pool must be large enough for accumulation to start but too small for - // sort_memory after clearing. - let task_ctx = new_spill_ctx(1, 500); + // Pool must be large enough for accumulation to start but too small + // for sort_memory after clearing. + let task_ctx = new_spill_ctx(1, max_memory, enable_blocked_groups); let result = collect(aggr.execute(0, Arc::clone(&task_ctx))?).await; match &result { @@ -4651,6 +4808,10 @@ mod tests { Ok(()) } + // ========================================================================= + // Multi-stage aggregation tests + // ========================================================================= + /// Tests that PartialReduce mode: /// 1. Accepts state as input (like Final) /// 2. Produces state as output (like Partial) @@ -4660,116 +4821,118 @@ mod tests { /// Partial -> PartialReduce -> Final #[tokio::test] async fn test_partial_reduce_mode() -> Result<()> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::UInt32, false), - Field::new("b", DataType::Float64, false), - ])); - - // Produce two partitions of input data - let batch1 = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(UInt32Array::from(vec![1, 2, 3])), - Arc::new(Float64Array::from(vec![10.0, 20.0, 30.0])), - ], - )?; - let batch2 = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(UInt32Array::from(vec![1, 2, 3])), - Arc::new(Float64Array::from(vec![40.0, 50.0, 60.0])), - ], - )?; + for enable_blocked in [false, true] { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::UInt32, false), + Field::new("b", DataType::Float64, false), + ])); + + // Produce two partitions of input data + let batch1 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(UInt32Array::from(vec![1, 2, 3])), + Arc::new(Float64Array::from(vec![10.0, 20.0, 30.0])), + ], + )?; + let batch2 = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(UInt32Array::from(vec![1, 2, 3])), + Arc::new(Float64Array::from(vec![40.0, 50.0, 60.0])), + ], + )?; - let groups = - PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); - let aggregates: Vec> = vec![Arc::new( - AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) - .schema(Arc::clone(&schema)) - .alias("SUM(b)") - .build()?, - )]; + let groups = + PhysicalGroupBy::new_single(vec![(col("a", &schema)?, "a".to_string())]); + let aggregates: Vec> = vec![Arc::new( + AggregateExprBuilder::new(sum_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias("SUM(b)") + .build()?, + )]; + + // Step 1: Partial aggregation on partition 1 + let input1 = + TestMemoryExec::try_new_exec(&[vec![batch1]], Arc::clone(&schema), None)?; + let partial1 = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + groups.clone(), + aggregates.clone(), + vec![None], + input1, + Arc::clone(&schema), + )?); - // Step 1: Partial aggregation on partition 1 - let input1 = - TestMemoryExec::try_new_exec(&[vec![batch1]], Arc::clone(&schema), None)?; - let partial1 = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - groups.clone(), - aggregates.clone(), - vec![None], - input1, - Arc::clone(&schema), - )?); + // Step 2: Partial aggregation on partition 2 + let input2 = + TestMemoryExec::try_new_exec(&[vec![batch2]], Arc::clone(&schema), None)?; + let partial2 = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + groups.clone(), + aggregates.clone(), + vec![None], + input2, + Arc::clone(&schema), + )?); - // Step 2: Partial aggregation on partition 2 - let input2 = - TestMemoryExec::try_new_exec(&[vec![batch2]], Arc::clone(&schema), None)?; - let partial2 = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - groups.clone(), - aggregates.clone(), - vec![None], - input2, - Arc::clone(&schema), - )?); + // Collect partial results + let task_ctx = task_ctx_with_blocked_groups(enable_blocked); + let partial_result1 = + crate::collect(Arc::clone(&partial1) as _, Arc::clone(&task_ctx)).await?; + let partial_result2 = + crate::collect(Arc::clone(&partial2) as _, Arc::clone(&task_ctx)).await?; - // Collect partial results - let task_ctx = Arc::new(TaskContext::default()); - let partial_result1 = - crate::collect(Arc::clone(&partial1) as _, Arc::clone(&task_ctx)).await?; - let partial_result2 = - crate::collect(Arc::clone(&partial2) as _, Arc::clone(&task_ctx)).await?; - - // The partial results have state schema (group cols + accumulator state) - let partial_schema = partial1.schema(); - - // Step 3: PartialReduce — combine partial results, still producing state - let combined_input = TestMemoryExec::try_new_exec( - &[partial_result1, partial_result2], - Arc::clone(&partial_schema), - None, - )?; - // Coalesce into a single partition for the PartialReduce - let coalesced = Arc::new(CoalescePartitionsExec::new(combined_input)); + // The partial results have state schema (group cols + accumulator state) + let partial_schema = partial1.schema(); - let partial_reduce = Arc::new(AggregateExec::try_new( - AggregateMode::PartialReduce, - groups.clone(), - aggregates.clone(), - vec![None], - coalesced, - Arc::clone(&partial_schema), - )?); + // Step 3: PartialReduce — combine partial results, still producing state + let combined_input = TestMemoryExec::try_new_exec( + &[partial_result1, partial_result2], + Arc::clone(&partial_schema), + None, + )?; + // Coalesce into a single partition for the PartialReduce + let coalesced = Arc::new(CoalescePartitionsExec::new(combined_input)); + + let partial_reduce = Arc::new(AggregateExec::try_new( + AggregateMode::PartialReduce, + groups.clone(), + aggregates.clone(), + vec![None], + coalesced, + Arc::clone(&partial_schema), + )?); - // Verify PartialReduce output schema matches Partial output schema - // (both produce state, not final values) - assert_eq!(partial_reduce.schema(), partial_schema); + // Verify PartialReduce output schema matches Partial output schema + // (both produce state, not final values) + assert_eq!(partial_reduce.schema(), partial_schema); - // Collect PartialReduce results - let reduce_result = - crate::collect(Arc::clone(&partial_reduce) as _, Arc::clone(&task_ctx)) - .await?; + // Collect PartialReduce results + let reduce_result = + crate::collect(Arc::clone(&partial_reduce) as _, Arc::clone(&task_ctx)) + .await?; - // Step 4: Final aggregation on the PartialReduce output - let final_input = TestMemoryExec::try_new_exec( - &[reduce_result], - Arc::clone(&partial_schema), - None, - )?; - let final_agg = Arc::new(AggregateExec::try_new( - AggregateMode::Final, - groups.clone(), - aggregates.clone(), - vec![None], - final_input, - Arc::clone(&partial_schema), - )?); + // Step 4: Final aggregation on the PartialReduce output + let final_input = TestMemoryExec::try_new_exec( + &[reduce_result], + Arc::clone(&partial_schema), + None, + )?; + let final_agg = Arc::new(AggregateExec::try_new( + AggregateMode::Final, + groups.clone(), + aggregates.clone(), + vec![None], + final_input, + Arc::clone(&partial_schema), + )?); - let result = crate::collect(final_agg, Arc::clone(&task_ctx)).await?; + let result = crate::collect(final_agg, Arc::clone(&task_ctx)).await?; - // Expected: group 1 -> 10+40=50, group 2 -> 20+50=70, group 3 -> 30+60=90 - assert_snapshot!(batches_to_sort_string(&result), @r" + // Expected: group 1 -> 10+40=50, group 2 -> 20+50=70, group 3 -> 30+60=90 + allow_duplicates! { + assert_snapshot!(batches_to_sort_string(&result), @r" +---+--------+ | a | SUM(b) | +---+--------+ @@ -4778,6 +4941,8 @@ mod tests { | 3 | 90.0 | +---+--------+ "); + } + } Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 97fbd519c825c..ebfdd4323984c 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -79,6 +79,7 @@ impl GroupOrdering { self.emit_to().map(|emit_to| match emit_to { EmitTo::First(max) => EmitTo::First(n.min(max)), EmitTo::All => EmitTo::First(n), + EmitTo::NextBlock => EmitTo::NextBlock, }) } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 056a7f171a516..e3552c30d7f80 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -43,7 +43,7 @@ use datafusion_common::{ }; use datafusion_execution::TaskContext; use datafusion_execution::memory_pool::proxy::VecAllocExt; -use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryLimit, MemoryReservation}; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::aggregate::AggregateFunctionExpr; use datafusion_physical_expr::expressions::Column; @@ -64,6 +64,11 @@ pub(crate) enum ExecutionState { /// When producing output, the remaining rows to output are stored /// here and are sliced off as needed in batch_size chunks ProducingOutput(RecordBatch), + /// Producing output block by block. + /// + /// It is the blocked version `ProducingOutput` and will be used + /// when blocked optimization is enabled. + ProducingBlocks, /// Produce intermediate aggregate state for each input row without /// aggregation. /// @@ -353,6 +358,35 @@ enum OutOfMemoryMode { /// │ 2 │ 2 │ 3.0 │ │ 2 │ 2 │ 3.0 │ └────────────┘ /// └─────────────────┘ └─────────────────┘ /// ``` +/// +/// # Blocked approach for intermediate results +/// +/// An important optimization for [`group_values`] and [`accumulators`] +/// is to manage such intermediate results using the blocked approach. +/// +/// In the original method, intermediate results are managed within a single large block +/// (can think of it as a Vec). As this block grows, it often triggers numerous +/// copies, resulting in poor performance. +/// +/// In contrast, the blocked approach allocates capacity for the block +/// based on a predefined block size firstly. +/// And when the block reaches its limit, we allocate a new block +/// (also with the same predefined block size based capacity) +/// instead of expanding the current one and copying the data. +/// This method eliminates unnecessary copies and significantly improves performance. +/// +/// You can find some implementation details(like how to locate data in such two approaches) +/// in [`GroupsAccumulator::supports_blocked_groups`] and [`GroupValues::supports_blocked_groups`]. +/// +/// And for a really detailed introduction to the design of blocked approach, maybe you can see [#7065]. +/// +/// The conditions that trigger the blocked groups optimization can be found in +/// [`can_enable_blocked_groups`]. +/// +/// [`group_values`]: Self::group_values +/// [`accumulators`]: Self::accumulators +/// [#7065]: +/// pub(crate) struct GroupedHashAggregateStream { // ======================================================================== // PROPERTIES: @@ -437,6 +471,9 @@ pub(crate) struct GroupedHashAggregateStream { /// current stream. skip_aggregation_probe: Option, + /// Have we enabled the blocked optimization for group values and accumulators + enable_blocked_groups: bool, + // ======================================================================== // EXECUTION RESOURCES: // Fields related to managing execution resources and monitoring performance. @@ -498,7 +535,7 @@ impl GroupedHashAggregateStream { }; // Instantiate the accumulators - let accumulators: Vec<_> = aggregate_exprs + let mut accumulators: Vec<_> = aggregate_exprs .iter() .map(create_group_accumulator) .collect::>()?; @@ -566,13 +603,15 @@ impl GroupedHashAggregateStream { .join(", "); let name = format!("GroupedHashAggregateStream[{partition}] ({agg_fn_names})"); let group_ordering = GroupOrdering::try_new(&agg.input_order_mode)?; - let oom_mode = match (agg.mode, &group_ordering) { + let no_memory_limit = + matches!(context.memory_pool().memory_limit(), MemoryLimit::Infinite); + let oom_mode = match (agg.mode, &group_ordering, no_memory_limit) { // In partial aggregation mode, always prefer to emit incomplete results early. - (AggregateMode::Partial, _) => OutOfMemoryMode::EmitEarly, + (AggregateMode::Partial, _, false) => OutOfMemoryMode::EmitEarly, // For non-partial aggregation modes, emitting incomplete results is not an option. // Instead, use disk spilling to store sorted, incomplete results, and merge them // afterwards. - (_, GroupOrdering::None | GroupOrdering::Partial(_)) + (_, GroupOrdering::None | GroupOrdering::Partial(_), false) if context.runtime_env().disk_manager.tmp_files_enabled() => { OutOfMemoryMode::Spill @@ -584,10 +623,11 @@ impl GroupedHashAggregateStream { // Therefore, we fall back to simply reporting the error immediately. // This mode will also be used if the `DiskManager` is not configured to allow spilling // to disk. + // Also if no memory limit in memory pool, but oom happens, it should be an error. _ => OutOfMemoryMode::ReportError, }; - let group_values = new_group_values(group_schema, &group_ordering)?; + let mut group_values = new_group_values(group_schema, &group_ordering)?; let reservation = MemoryConsumer::new(name) // We interpret 'can spill' as 'can handle memory back pressure'. // This value needs to be set to true for the default memory pool implementations @@ -649,6 +689,29 @@ impl GroupedHashAggregateStream { None }; + // Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. + let can_enable_blocked_groups = can_enable_blocked_groups( + &group_ordering, + &oom_mode, + &group_values, + &accumulators, + ); + + let enable_blocked_groups = context + .session_config() + .options() + .execution + .enable_aggregation_blocked_groups + && can_enable_blocked_groups; + + if enable_blocked_groups { + group_values.alter_block_size(Some(batch_size))?; + accumulators + .iter_mut() + .try_for_each(|acc| acc.alter_block_size(Some(batch_size)))?; + } + + // Metrics for aggregation stats let reduction_factor = if agg.mode == AggregateMode::Partial { Some( MetricBuilder::new(&agg.metrics) @@ -680,6 +743,7 @@ impl GroupedHashAggregateStream { spill_state, group_values_soft_limit: agg.limit_options().map(|config| config.limit()), skip_aggregation_probe, + enable_blocked_groups, reduction_factor, }) } @@ -705,6 +769,69 @@ pub(crate) fn create_group_accumulator( } } +/// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. +/// The blocked optimization can be enabled when: +/// - It is not streaming aggregation(because blocked mode can't support Emit::first(exact n)) +/// - The spilling is disabled(still need to consider more to support it efficiently) +/// - [`GroupValues::supports_blocked_groups`] and all [`GroupsAccumulator::supports_blocked_groups`] are true +/// +/// [`GroupValues::supports_blocked_groups`]: crate::aggregates::group_values::GroupValues::supports_blocked_groups +/// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr::GroupsAccumulator::supports_blocked_groups +/// +// TODO: support blocked optimization in streaming, spilling, and maybe empty accumulators case? +#[expect(clippy::borrowed_box)] +fn can_enable_blocked_groups( + group_ordering: &GroupOrdering, + oom_mode: &OutOfMemoryMode, + group_values: &Box, + accumulators: &[Box], +) -> bool { + if !matches!(group_ordering, GroupOrdering::None) + || !matches!(oom_mode, OutOfMemoryMode::ReportError) + { + return false; + } + + let group_values_supports_blocked = group_values.supports_blocked_groups(); + let accumulators_support_blocked = + accumulators.iter().all(|acc| acc.supports_blocked_groups()); + group_values_supports_blocked && accumulators_support_blocked +} + +// fn maybe_enable_blocked_groups( +// context: &TaskContext, +// group_values: &mut dyn GroupValues, +// accumulators: &mut [Box], +// block_size: usize, +// group_ordering: &GroupOrdering, +// ) -> Result { +// if !context +// .session_config() +// .options() +// .execution +// .enable_aggregation_blocked_groups +// || !matches!(group_ordering, GroupOrdering::None) +// || !matches!(context.memory_pool().memory_limit(), MemoryLimit::Infinite) +// { +// return Ok(false); +// } + +// let group_values_supports_blocked = group_values.supports_blocked_groups(); +// let accumulators_support_blocked = +// accumulators.iter().all(|acc| acc.supports_blocked_groups()); + +// match (group_values_supports_blocked, accumulators_support_blocked) { +// (true, true) => { +// group_values.alter_block_size(Some(block_size))?; +// accumulators +// .iter_mut() +// .try_for_each(|acc| acc.alter_block_size(Some(block_size)))?; +// Ok(true) +// } +// _ => Ok(false), +// } +// } + impl Stream for GroupedHashAggregateStream { type Item = Result; @@ -880,6 +1007,33 @@ impl Stream for GroupedHashAggregateStream { ))); } + ExecutionState::ProducingBlocks => { + // Try to emit and then: + // - If found `Err`, throw it, end this stream abnormally + // - If found `None`, it means all blocks are polled, end this stream normally + // - If found `Some`, return it and wait next polling + let emit_result = self.emit(EmitTo::NextBlock, false); + let Ok(batch_opt) = emit_result else { + return Poll::Ready(Some(Err(emit_result.unwrap_err()))); + }; + + let Some(batch) = batch_opt else { + self.exec_state = if self.input_done { + ExecutionState::Done + } else if self.should_skip_aggregation() { + ExecutionState::SkippingAggregation + } else { + ExecutionState::ReadingInput + }; + continue; + }; + + debug_assert!(batch.num_rows() > 0); + return Poll::Ready(Some(Ok( + batch.record_output(&self.baseline_metrics) + ))); + } + ExecutionState::Done => { // Sanity check: all groups should have been emitted by now if !self.group_values.is_empty() { @@ -1223,12 +1377,21 @@ impl GroupedHashAggregateStream { let timer = elapsed_compute.timer(); self.exec_state = if self.spill_state.spills.is_empty() { // Input has been entirely processed without spilling to disk. + if !self.enable_blocked_groups { + // Flush any remaining group values. + let batch = self.emit(EmitTo::All, false)?; - // Flush any remaining group values. - let batch = self.emit(EmitTo::All, false)?; - - // If there are none, we're done; otherwise switch to emitting them - batch.map_or(ExecutionState::Done, ExecutionState::ProducingOutput) + // If there are none, we're done; otherwise switch to emitting them + batch.map_or(ExecutionState::Done, ExecutionState::ProducingOutput) + } else { + assert!(can_enable_blocked_groups( + &self.group_ordering, + &self.oom_mode, + &self.group_values, + &self.accumulators + )); + ExecutionState::ProducingBlocks + } } else { // Spill any remaining data to disk. There is some performance overhead in // writing out this last chunk of data and reading it back. The benefit of @@ -1305,10 +1468,21 @@ impl GroupedHashAggregateStream { fn switch_to_skip_aggregation(&mut self) -> Result> { if let Some(probe) = self.skip_aggregation_probe.as_mut() && probe.should_skip() - && let Some(batch) = self.emit(EmitTo::All, false)? { - return Ok(Some(ExecutionState::ProducingOutput(batch))); - }; + if !self.enable_blocked_groups { + if let Some(batch) = self.emit(EmitTo::All, false)? { + return Ok(Some(ExecutionState::ProducingOutput(batch))); + }; + } else { + assert!(can_enable_blocked_groups( + &self.group_ordering, + &self.oom_mode, + &self.group_values, + &self.accumulators + )); + return Ok(Some(ExecutionState::ProducingBlocks)); + } + } Ok(None) } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index ab8a4a293234e..dc06461d83efa 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -217,6 +217,7 @@ datafusion.catalog.newlines_in_values false datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics true +datafusion.execution.enable_aggregation_blocked_groups true datafusion.execution.enable_ansi_mode false datafusion.execution.enable_recursive_ctes true datafusion.execution.enforce_batch_size_in_joins false @@ -362,6 +363,7 @@ datafusion.catalog.newlines_in_values false Specifies whether newlines in (quote datafusion.execution.batch_size 8192 Default batch size while creating new batches, it's especially useful for buffer-in-memory batches since creating tiny batches would result in too much metadata memory consumption datafusion.execution.coalesce_batches true When set to true, record batches will be examined between each operator and small batches will be coalesced into larger batches. This is helpful when there are highly selective filters or joins that could produce tiny output batches. The target batch size is determined by the configuration setting datafusion.execution.collect_statistics true Should DataFusion collect statistics when first creating a table. Has no effect after the table is created. Applies to the default `ListingTableProvider` in DataFusion. Defaults to true. +datafusion.execution.enable_aggregation_blocked_groups true Should DataFusion use a blocked approach to manage grouping state. By default, the blocked approach is used which allocates capacity based on a predefined block size firstly. When the block reaches its limit, we allocate a new block (also with the same predefined block size based capacity) instead of expanding the current one and copying the data. If `false`, a single allocation approach is used, where values are managed within a single large memory block. As this block grows, it often triggers numerous copies, resulting in poor performance. datafusion.execution.enable_ansi_mode false Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. datafusion.execution.enable_recursive_ctes true Should DataFusion support recursive CTEs datafusion.execution.enforce_batch_size_in_joins false Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index b828f0e793d47..18eae02e43104 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -134,6 +134,7 @@ The following configuration settings are available: | datafusion.execution.use_row_number_estimates_to_optimize_partitioning | false | Should DataFusion use row number estimates at the input to decide whether increasing parallelism is beneficial or not. By default, only exact row numbers (not estimates) are used for this decision. Setting this flag to `true` will likely produce better plans. if the source of statistics is accurate. We plan to make this the default in the future. | | datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | | datafusion.execution.objectstore_writer_buffer_size | 10485760 | Size (bytes) of data buffer DataFusion uses when writing output files. This affects the size of the data chunks that are uploaded to remote object stores (e.g. AWS S3). If very large (>= 100 GiB) output files are being written, it may be necessary to increase this size to avoid errors from the remote end point. | +| datafusion.execution.enable_aggregation_blocked_groups | true | Should DataFusion use a blocked approach to manage grouping state. By default, the blocked approach is used which allocates capacity based on a predefined block size firstly. When the block reaches its limit, we allocate a new block (also with the same predefined block size based capacity) instead of expanding the current one and copying the data. If `false`, a single allocation approach is used, where values are managed within a single large memory block. As this block grows, it often triggers numerous copies, resulting in poor performance. | | datafusion.execution.enable_ansi_mode | false | Whether to enable ANSI SQL mode. The flag is experimental and relevant only for DataFusion Spark built-in functions When `enable_ansi_mode` is set to `true`, the query engine follows ANSI SQL semantics for expressions, casting, and error handling. This means: - **Strict type coercion rules:** implicit casts between incompatible types are disallowed. - **Standard SQL arithmetic behavior:** operations such as division by zero, numeric overflow, or invalid casts raise runtime errors rather than returning `NULL` or adjusted values. - **Consistent ANSI behavior** for string concatenation, comparisons, and `NULL` handling. When `enable_ansi_mode` is `false` (the default), the engine uses a more permissive, non-ANSI mode designed for user convenience and backward compatibility. In this mode: - Implicit casts between types are allowed (e.g., string to integer when possible). - Arithmetic operations are more lenient — for example, `abs()` on the minimum representable integer value returns the input value instead of raising overflow. - Division by zero or invalid casts may return `NULL` instead of failing. # Default `false` — ANSI SQL mode is disabled by default. | | datafusion.execution.hash_join_buffering_capacity | 0 | How many bytes to buffer in the probe side of hash joins while the build side is concurrently being built. Without this, hash joins will wait until the full materialization of the build side before polling the probe side. This is useful in scenarios where the query is not completely CPU bounded, allowing to do some early work concurrently and reducing the latency of the query. Note that when hash join buffering is enabled, the probe side will start eagerly polling data, not giving time for the producer side of dynamic filters to produce any meaningful predicate. Queries with dynamic filters might see performance degradation. Disabled by default, set to a number greater than 0 for enabling it. | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. |