From 43537485df6d1dd02a8bbf0a27f005454f75ff09 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 9 Apr 2025 15:47:58 +0800 Subject: [PATCH 01/55] define the needed methods in `GroupAccumulator` and `GroupValues`. --- .../expr-common/src/groups_accumulator.rs | 28 ++++++++++++++++++- .../src/aggregates/group_values/mod.rs | 26 +++++++++++++++++ 2 files changed, 53 insertions(+), 1 deletion(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 5ff1c1d07216..d0e990807c93 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -18,7 +18,7 @@ //! Vectorized [`GroupsAccumulator`] use arrow::array::{ArrayRef, BooleanArray}; -use datafusion_common::{not_impl_err, Result}; +use datafusion_common::{not_impl_err, DataFusionError, Result}; /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy)] @@ -250,4 +250,30 @@ pub trait GroupsAccumulator: Send { /// 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. + 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/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index ce56ca4f7dfd..c71194de6099 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -110,6 +110,32 @@ pub(crate) 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, batch: &RecordBatch); + + /// Returns `true` if this accumulator supports blocked groups. + 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. From a4450f7ea23a0bd4cbb23d25b8bb538ec9fd2a51 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 9 Apr 2025 22:36:04 +0800 Subject: [PATCH 02/55] define `GroupIndexOperations`, and impl block/flat mode for it. --- .../src/aggregate/groups_accumulator.rs | 78 +++++++++++++++++++ 1 file changed, 78 insertions(+) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index aa2f5a586e87..6674a5551f92 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -507,3 +507,81 @@ pub(crate) fn slice_and_maybe_filter( Ok(sliced_arrays) } } + +/// Blocked style group index used in blocked mode group values and accumulators +/// - High 32 bits represent `block_id` +/// - Low 32 bits represent `block_offset` +/// +pub trait GroupIndexOperations { + 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; +} + +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 + } +} + +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 + } +} + +// #[cfg(test)] +// mod tests { +// use super::*; + +// #[test] +// fn test_blocked_group_index_build() { +// let group_index1 = 1; +// let group_index2 = (42_u64 << 32) | 2; +// let group_index3 = ((u32::MAX as u64) << 32) | 3; + +// let index_builder = BlockedGroupIndexBuilder::new(false); +// let flat1 = index_builder.build(group_index1 as usize); +// let flat2 = index_builder.build(group_index2 as usize); +// let flat3 = index_builder.build(group_index3 as usize); +// let expected1 = BlockedGroupIndex::new_from_parts(0, group_index1); +// let expected2 = BlockedGroupIndex::new_from_parts(0, group_index2); +// let expected3 = BlockedGroupIndex::new_from_parts(0, group_index3); +// assert_eq!(flat1, expected1); +// assert_eq!(flat2, expected2); +// assert_eq!(flat3, expected3); + +// let index_builder = BlockedGroupIndexBuilder::new(true); +// let blocked1 = index_builder.build(group_index1 as usize); +// let blocked2 = index_builder.build(group_index2 as usize); +// let blocked3 = index_builder.build(group_index3 as usize); +// let expected1 = BlockedGroupIndex::new_from_parts(0, 1); +// let expected2 = BlockedGroupIndex::new_from_parts(42, 2); +// let expected3 = BlockedGroupIndex::new_from_parts(u32::MAX, 3); +// assert_eq!(blocked1, expected1); +// assert_eq!(blocked2, expected2); +// assert_eq!(blocked3, expected3); +// } +// } From 96b74359820ee0797cddeb359718f251b16fdd79 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 9 Apr 2025 22:37:04 +0800 Subject: [PATCH 03/55] support block approach for `GroupValuesPrimitive`. --- Cargo.lock | 1 + datafusion/physical-plan/Cargo.toml | 1 + .../src/aggregates/group_values/mod.rs | 2 +- .../group_values/single_group_by/primitive.rs | 252 ++++++++++++------ 4 files changed, 174 insertions(+), 82 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8c501a873133..1144c72c283d 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2480,6 +2480,7 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-functions-aggregate", + "datafusion-functions-aggregate-common", "datafusion-functions-window", "datafusion-functions-window-common", "datafusion-physical-expr", diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 5210ee26755c..3182ba119796 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -51,6 +51,7 @@ datafusion-common = { workspace = true, default-features = true } datafusion-common-runtime = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } datafusion-functions-window-common = { workspace = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-physical-expr-common = { workspace = true } diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index c71194de6099..a4d91933b5a7 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::{downcast_primitive, ArrayRef, RecordBatch}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; use datafusion_expr::EmitTo; 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 d945d3ddcbf5..63b57821d720 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 @@ -27,8 +27,12 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, +}; use half::f16; use hashbrown::hash_table::HashTable; +use std::collections::VecDeque; use std::mem::size_of; use std::sync::Arc; @@ -81,17 +85,31 @@ hash_float!(f16, f32, f64); pub struct GroupValuesPrimitive { /// The data type of the output array data_type: DataType, + /// Stores the group index based on the hash of its value /// /// We don't store the hashes as hashing fixed width primitives /// is fast enough for this not to benefit performance - map: HashTable, + map: HashTable, + /// 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 { @@ -100,9 +118,10 @@ impl GroupValuesPrimitive { Self { data_type, map: HashTable::with_capacity(128), - values: Vec::with_capacity(128), + values: VecDeque::new(), null_group: None, random_state: Default::default(), + block_size: None, } } } @@ -112,43 +131,30 @@ where T::Native: HashValue, { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { - assert_eq!(cols.len(), 1); - groups.clear(); - - for v in cols[0].as_primitive::() { - let group_id = match v { - None => *self.null_group.get_or_insert_with(|| { - let group_id = self.values.len(); - self.values.push(Default::default()); - group_id - }), - Some(key) => { - let state = &self.random_state; - let hash = key.hash(state); - let insert = self.map.entry( - hash, - |g| unsafe { self.values.get_unchecked(*g).is_eq(key) }, - |g| unsafe { self.values.get_unchecked(*g).hash(state) }, - ); - - match insert { - hashbrown::hash_table::Entry::Occupied(o) => *o.get(), - hashbrown::hash_table::Entry::Vacant(v) => { - let g = self.values.len(); - v.insert(g); - self.values.push(key); - g - } - } + if let Some(block_size) = self.block_size { + let before_add_group = |group_values: &mut VecDeque>| { + if group_values.back().unwrap().len() == block_size { + let new_block = Vec::with_capacity(block_size); + group_values.push_back(new_block); } }; - groups.push(group_id) + self.get_or_create_groups::<_, BlockedGroupIndexOperations>( + cols, + groups, + before_add_group, + ) + } else { + self.get_or_create_groups::<_, FlatGroupIndexOperations>( + cols, + groups, + |_: &mut VecDeque>| {}, + ) } - Ok(()) } fn size(&self) -> usize { - self.map.capacity() * size_of::() + self.values.allocated_size() + todo!() + // self.map.capacity() * size_of::() + self.values.len } fn is_empty(&self) -> bool { @@ -160,54 +166,55 @@ where } fn emit(&mut self, emit_to: EmitTo) -> Result> { - fn build_primitive( - values: Vec, - null_idx: Option, - ) -> PrimitiveArray { - let nulls = null_idx.map(|null_idx| { - let mut buffer = NullBufferBuilder::new(values.len()); - buffer.append_n_non_nulls(null_idx); - buffer.append_null(); - buffer.append_n_non_nulls(values.len() - null_idx - 1); - // NOTE: The inner builder must be constructed as there is at least one null - buffer.finish().unwrap() - }); - PrimitiveArray::::new(values.into(), nulls) - } + todo!() + // fn build_primitive( + // values: Vec, + // null_idx: Option, + // ) -> PrimitiveArray { + // let nulls = null_idx.map(|null_idx| { + // let mut buffer = NullBufferBuilder::new(values.len()); + // buffer.append_n_non_nulls(null_idx); + // buffer.append_null(); + // buffer.append_n_non_nulls(values.len() - null_idx - 1); + // // NOTE: The inner builder must be constructed as there is at least one null + // buffer.finish().unwrap() + // }); + // PrimitiveArray::::new(values.into(), nulls) + // } - let array: PrimitiveArray = match emit_to { - EmitTo::All => { - self.map.clear(); - build_primitive(std::mem::take(&mut self.values), self.null_group.take()) - } - EmitTo::First(n) => { - self.map.retain(|group_idx| { - // Decrement group index by n - match group_idx.checked_sub(n) { - // Group index was >= n, shift value down - Some(sub) => { - *group_idx = sub; - true - } - // Group index was < n, so remove from table - None => false, - } - }); - let null_group = match &mut self.null_group { - Some(v) if *v >= n => { - *v -= n; - None - } - Some(_) => self.null_group.take(), - None => None, - }; - let mut split = self.values.split_off(n); - std::mem::swap(&mut self.values, &mut split); - build_primitive(split, null_group) - } - }; + // let array: PrimitiveArray = match emit_to { + // EmitTo::All => { + // self.map.clear(); + // build_primitive(std::mem::take(&mut self.values), self.null_group.take()) + // } + // EmitTo::First(n) => { + // self.map.retain(|group_idx| { + // // Decrement group index by n + // match group_idx.checked_sub(n) { + // // Group index was >= n, shift value down + // Some(sub) => { + // *group_idx = sub; + // true + // } + // // Group index was < n, so remove from table + // None => false, + // } + // }); + // let null_group = match &mut self.null_group { + // Some(v) if *v >= n => { + // *v -= n; + // None + // } + // Some(_) => self.null_group.take(), + // None => None, + // }; + // let mut split = self.values.split_off(n); + // std::mem::swap(&mut self.values, &mut split); + // build_primitive(split, null_group) + // } + // }; - Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) + // Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) } fn clear_shrink(&mut self, batch: &RecordBatch) { @@ -218,3 +225,86 @@ where self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared } } + +impl GroupValuesPrimitive +where + T::Native: HashValue, +{ + fn get_or_create_groups( + &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_index = match v { + None => *self.null_group.get_or_insert_with(|| { + // actions before add new group like checking if room is enough + before_add_group(&mut self.values); + + // get block infos and update block + let block_id = self.values.len() as u32; + 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| unsafe { + let block_id = O::get_block_id(*g); + let block_offset = O::get_block_offset(*g); + self.values + .get(block_id as usize) + .unwrap() + .get_unchecked(block_offset as usize) + .is_eq(key) + }, + |g| unsafe { + let block_id = O::get_block_id(*g); + let block_offset = O::get_block_offset(*g); + self.values + .get(block_id as usize) + .unwrap() + .get_unchecked(block_offset as usize) + .hash(state) + }, + ); + + match insert { + hashbrown::hash_table::Entry::Occupied(o) => *o.get(), + hashbrown::hash_table::Entry::Vacant(v) => { + // actions before add new group like checking if room is enough + before_add_group(&mut self.values); + + // get block infos and update block + let block_id = self.values.len() as u32; + 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); + packed_index + } + } + } + }; + groups.push(group_index as usize) + } + Ok(()) + } +} From 38d4fc61b7f1454a81e63632255d960c3c830eef Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 10 Apr 2025 19:06:48 +0800 Subject: [PATCH 04/55] add new emit mode. --- datafusion-examples/examples/advanced_udaf.rs | 8 ++-- .../expr-common/src/groups_accumulator.rs | 44 ++++++++++++++++++- .../src/aggregate/groups_accumulator.rs | 4 +- .../groups_accumulator/accumulate.rs | 1 + .../aggregate/groups_accumulator/bool_op.rs | 3 ++ .../aggregate/groups_accumulator/prim_op.rs | 2 +- datafusion/functions-aggregate/src/average.rs | 8 ++-- .../functions-aggregate/src/correlation.rs | 6 +++ datafusion/functions-aggregate/src/count.rs | 4 +- .../functions-aggregate/src/first_last.rs | 12 +++-- datafusion/functions-aggregate/src/median.rs | 4 +- .../src/min_max/min_max_bytes.rs | 3 ++ .../functions-aggregate/src/variance.rs | 12 ++--- .../group_values/multi_group_by/mod.rs | 3 ++ .../src/aggregates/group_values/row.rs | 3 ++ .../group_values/single_group_by/bytes.rs | 3 ++ .../single_group_by/bytes_view.rs | 3 ++ .../group_values/single_group_by/primitive.rs | 5 ++- 18 files changed, 102 insertions(+), 26 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 9cda726db719..f38b187a6881 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -307,8 +307,8 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { /// Generate output, as specified by `emit_to` and update the intermediate state fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed(&mut self.counts); - let prods = emit_to.take_needed(&mut self.prods); + let counts = emit_to.take_needed_rows(&mut self.counts); + let prods = emit_to.take_needed_rows(&mut self.prods); let nulls = self.null_state.build(emit_to); assert_eq!(nulls.len(), prods.len()); @@ -346,10 +346,10 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); - let counts = emit_to.take_needed(&mut self.counts); + let counts = emit_to.take_needed_rows(&mut self.counts); let counts = UInt32Array::new(counts.into(), nulls.clone()); // zero copy - let prods = emit_to.take_needed(&mut self.prods); + let prods = emit_to.take_needed_rows(&mut self.prods); let prods = PrimitiveArray::::new(prods.into(), nulls) // zero copy .with_data_type(self.prod_data_type.clone()); diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index d0e990807c93..a06ae90f5076 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,6 +17,8 @@ //! Vectorized [`GroupsAccumulator`] +use std::collections::VecDeque; + use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -31,15 +33,40 @@ 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 + /// + /// The flag's meaning: + /// - `true` represents new groups still will be added, + /// and we need to shift the values down. + /// - `false` represents no new groups will be added again, + /// and we don't need to shift the values down. + NextBlock(bool), } impl EmitTo { + /// Remove and return `needed values` from `values`. + pub fn take_needed( + &self, + values: &mut VecDeque>, + is_blocked_groups: bool, + ) -> Vec { + if is_blocked_groups { + self.take_needed_block(values) + } else { + assert_eq!(values.len(), 1); + self.take_needed_rows(values.back_mut().unwrap()) + } + } + /// Removes the number of rows from `v` required to emit the right /// number of rows, returning a `Vec` with elements taken, and the /// remaining values in `v`. /// /// This avoids copying if Self::All - pub fn take_needed(&self, v: &mut Vec) -> Vec { + /// + /// NOTICE: only support emit strategies: `Self::All` and `Self::First` + /// + pub fn take_needed_rows(&self, v: &mut Vec) -> Vec { match self { Self::All => { // Take the entire vector, leave new (empty) vector @@ -52,8 +79,23 @@ impl EmitTo { std::mem::swap(v, &mut t); t } + Self::NextBlock(_) => unreachable!("don't support take block in take_needed"), } } + + /// Removes one block required to emit and return it + /// + /// NOTICE: only support emit strategy `Self::NextBlock` + /// + fn take_needed_block(&self, blocks: &mut VecDeque>) -> Vec { + assert!( + matches!(self, Self::NextBlock(_)), + "only support take block in take_needed_block" + ); + blocks + .pop_front() + .expect("should not call emit for empty blocks") + } } /// `GroupsAccumulator` implements a single aggregate (e.g. AVG) and diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 6674a5551f92..98f6238a6a1d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -321,7 +321,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { fn evaluate(&mut self, emit_to: EmitTo) -> Result { let vec_size_pre = self.states.allocated_size(); - let states = emit_to.take_needed(&mut self.states); + let states = emit_to.take_needed_rows(&mut self.states); let results: Vec = states .into_iter() @@ -341,7 +341,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { // filtered_null_mask(opt_filter, &values); fn state(&mut self, emit_to: EmitTo) -> Result> { let vec_size_pre = self.states.allocated_size(); - let states = emit_to.take_needed(&mut self.states); + let states = emit_to.take_needed_rows(&mut self.states); // each accumulator produces a potential vector of values // which we need to form into columns 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 e629e99e1657..62ff887d446f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -228,6 +228,7 @@ impl NullState { } first_n_null } + EmitTo::NextBlock(_) => todo!(), }; NullBuffer::new(nulls) } 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 149312e5a9c0..e661e0510d37 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 @@ -117,6 +117,9 @@ where } first_n } + EmitTo::NextBlock(_) => { + unreachable!("this accumulator still not support blocked groups") + } }; let nulls = self.null_state.build(emit_to); 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 078982c983fc..d7b30b3a5775 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 @@ -115,7 +115,7 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = emit_to.take_needed(&mut self.values); + let values = emit_to.take_needed_rows(&mut self.values); let nulls = self.null_state.build(emit_to); let values = PrimitiveArray::::new(values.into(), Some(nulls)) // no copy .with_data_type(self.data_type.clone()); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 798a039f50b1..ba21a1fd0832 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -596,8 +596,8 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed(&mut self.counts); - let sums = emit_to.take_needed(&mut self.sums); + let counts = emit_to.take_needed_rows(&mut self.counts); + let sums = emit_to.take_needed_rows(&mut self.sums); let nulls = self.null_state.build(emit_to); assert_eq!(nulls.len(), sums.len()); @@ -636,10 +636,10 @@ where let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); - let counts = emit_to.take_needed(&mut self.counts); + let counts = emit_to.take_needed_rows(&mut self.counts); let counts = UInt64Array::new(counts.into(), nulls.clone()); // zero copy - let sums = emit_to.take_needed(&mut self.sums); + let sums = emit_to.take_needed_rows(&mut self.sums); let sums = PrimitiveArray::::new(sums.into(), nulls) // zero copy .with_data_type(self.sum_data_type.clone()); diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index ac57256ce882..51902092cf25 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -448,6 +448,9 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { let n = match emit_to { EmitTo::All => self.count.len(), EmitTo::First(n) => n, + EmitTo::NextBlock(_) => { + unreachable!("this accumulator still not support blocked groups") + } }; let mut values = Vec::with_capacity(n); @@ -501,6 +504,9 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { let n = match emit_to { EmitTo::All => self.count.len(), EmitTo::First(n) => n, + EmitTo::NextBlock(_) => { + unreachable!("this accumulator still not support blocked groups") + } }; Ok(vec![ diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 2d995b4a4179..1b88c9921023 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -537,7 +537,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed(&mut self.counts); + let counts = emit_to.take_needed_rows(&mut self.counts); // Count is always non null (null inputs just don't contribute to the overall values) let nulls = None; @@ -548,7 +548,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { // return arrays for counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let counts = emit_to.take_needed(&mut self.counts); + let counts = emit_to.take_needed_rows(&mut self.counts); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls Ok(vec![Arc::new(counts) as ArrayRef]) } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index ec8c440b77e5..5b4ae6566a31 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -428,7 +428,7 @@ where } fn take_orderings(&mut self, emit_to: EmitTo) -> Vec> { - let result = emit_to.take_needed(&mut self.orderings); + let result = emit_to.take_needed_rows(&mut self.orderings); match emit_to { EmitTo::All => self.size_of_orderings = 0, @@ -436,6 +436,9 @@ where self.size_of_orderings -= result.iter().map(ScalarValue::size_of_vec).sum::() } + EmitTo::NextBlock(_) => { + unreachable!("this accumulator still not support blocked groups") + } } result @@ -460,6 +463,9 @@ where } first_n } + EmitTo::NextBlock(_) => { + unreachable!("this group values still not support blocked groups") + } } } @@ -513,7 +519,7 @@ where &mut self, emit_to: EmitTo, ) -> (ArrayRef, Vec>, BooleanBuffer) { - emit_to.take_needed(&mut self.min_of_each_group_buf.0); + emit_to.take_needed_rows(&mut self.min_of_each_group_buf.0); self.min_of_each_group_buf .1 .truncate(self.min_of_each_group_buf.0.len()); @@ -615,7 +621,7 @@ where } fn take_vals_and_null_buf(&mut self, emit_to: EmitTo) -> ArrayRef { - let r = emit_to.take_needed(&mut self.vals); + let r = emit_to.take_needed_rows(&mut self.vals); let null_buf = NullBuffer::new(Self::take_need(&mut self.null_builder, emit_to)); diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index ba6b63260e06..32931f56c0ee 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -393,7 +393,7 @@ impl GroupsAccumulator for MedianGroupsAccumulator Result> { // Emit values - let emit_group_values = emit_to.take_needed(&mut self.group_values); + let emit_group_values = emit_to.take_needed_rows(&mut self.group_values); // Build offsets let mut offsets = Vec::with_capacity(self.group_values.len() + 1); @@ -432,7 +432,7 @@ impl GroupsAccumulator for MedianGroupsAccumulator Result { // Emit values - let emit_group_values = emit_to.take_needed(&mut self.group_values); + let emit_group_values = emit_to.take_needed_rows(&mut self.group_values); // Calculate median for each group let mut evaluate_result_builder = 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 05321c2ff52d..45871bbd20eb 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -511,6 +511,9 @@ impl MinMaxBytesState { self.total_data_bytes -= first_data_capacity; (first_data_capacity, first_min_maxes) } + EmitTo::NextBlock(_) => { + unreachable!("this accumulator still not support blocked groups") + } } } diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 53e3e0cc56cd..5947fbf2c81d 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -460,11 +460,11 @@ impl VarianceGroupsAccumulator { &mut self, emit_to: datafusion_expr::EmitTo, ) -> (Vec, NullBuffer) { - let mut counts = emit_to.take_needed(&mut self.counts); + let mut counts = emit_to.take_needed_rows(&mut self.counts); // means are only needed for updating m2s and are not needed for the final result. // But we still need to take them to ensure the internal state is consistent. - let _ = emit_to.take_needed(&mut self.means); - let m2s = emit_to.take_needed(&mut self.m2s); + let _ = emit_to.take_needed_rows(&mut self.means); + let m2s = emit_to.take_needed_rows(&mut self.m2s); if let StatsType::Sample = self.stats_type { counts.iter_mut().for_each(|count| { @@ -555,9 +555,9 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { } fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { - let counts = emit_to.take_needed(&mut self.counts); - let means = emit_to.take_needed(&mut self.means); - let m2s = emit_to.take_needed(&mut self.m2s); + let counts = emit_to.take_needed_rows(&mut self.counts); + let means = emit_to.take_needed_rows(&mut self.means); + let m2s = emit_to.take_needed_rows(&mut self.m2s); Ok(vec![ Arc::new(UInt64Array::new(counts.into(), None)), 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 ac96a98edfe1..bb8bd765c2ca 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 @@ -1154,6 +1154,9 @@ impl GroupValues for GroupValuesColumn { output } + EmitTo::NextBlock(_) => { + unreachable!("this group values still 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 63751d470313..8022ac637d1b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -229,6 +229,9 @@ impl GroupValues for GroupValuesRows { }); output } + EmitTo::NextBlock(_) => { + unreachable!("this group values still not support blocked groups") + } }; // TODO: Materialize dictionaries in group keys 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 9686b8c3521d..e3768b78d9b3 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 @@ -116,6 +116,9 @@ impl GroupValues for GroupValuesByes { emit_group_values } + EmitTo::NextBlock(_) => { + unreachable!("this group values still 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 be9a0334e3ee..551f4c2eb530 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 @@ -117,6 +117,9 @@ impl GroupValues for GroupValuesBytesView { emit_group_values } + EmitTo::NextBlock(_) => { + unreachable!("this group values still 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 63b57821d720..2bfc3f446a1d 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 @@ -115,10 +115,13 @@ pub struct GroupValuesPrimitive { impl GroupValuesPrimitive { pub fn new(data_type: DataType) -> Self { assert!(PrimitiveArray::::is_compatible(&data_type)); + let mut values = VecDeque::new(); + values.push_back(Vec::new()); + Self { data_type, map: HashTable::with_capacity(128), - values: VecDeque::new(), + values, null_group: None, random_state: Default::default(), block_size: None, From 72e0fc31d1bed1a447048fff44f196201622b729 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 10 Apr 2025 20:34:10 +0800 Subject: [PATCH 05/55] make `NullState` codes common for better reuse. --- datafusion-examples/examples/advanced_udaf.rs | 17 +- .../src/aggregate/groups_accumulator.rs | 5 +- .../groups_accumulator/accumulate.rs | 259 +++++++++++++++--- .../aggregate/groups_accumulator/bool_op.rs | 9 +- .../aggregate/groups_accumulator/prim_op.rs | 10 +- datafusion/functions-aggregate/src/average.rs | 17 +- datafusion/physical-expr/src/aggregate.rs | 4 +- datafusion/physical-expr/src/lib.rs | 2 +- 8 files changed, 253 insertions(+), 70 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index f38b187a6881..6d259a2dd371 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -16,7 +16,7 @@ // under the License. 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::{any::Any, sync::Arc}; @@ -217,7 +217,7 @@ struct GeometricMeanGroupsAccumulator { prods: Vec, /// Track nulls in the input / filters - null_state: NullState, + null_state: FlatNullState, } impl GeometricMeanGroupsAccumulator { @@ -227,7 +227,7 @@ impl GeometricMeanGroupsAccumulator { return_data_type: DataType::Float64, counts: vec![], prods: vec![], - null_state: NullState::new(), + null_state: FlatNullState::new(), } } } @@ -254,7 +254,8 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { 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); @@ -284,8 +285,8 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { 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; }, ); @@ -296,8 +297,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/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 98f6238a6a1d..f7aa27826334 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -23,6 +23,7 @@ pub mod bool_op; pub mod nulls; pub mod prim_op; +use std::fmt::Debug; use std::mem::{size_of, size_of_val}; use arrow::array::new_empty_array; @@ -512,7 +513,7 @@ pub(crate) fn slice_and_maybe_filter( /// - High 32 bits represent `block_id` /// - Low 32 bits represent `block_offset` /// -pub trait GroupIndexOperations { +pub trait GroupIndexOperations: Debug { fn pack_index(block_id: u32, block_offset: u64) -> u64; fn get_block_id(packed_index: u64) -> u32; @@ -520,6 +521,7 @@ pub trait GroupIndexOperations { fn get_block_offset(packed_index: u64) -> u64; } +#[derive(Debug)] pub struct BlockedGroupIndexOperations; impl GroupIndexOperations for BlockedGroupIndexOperations { @@ -536,6 +538,7 @@ impl GroupIndexOperations for BlockedGroupIndexOperations { } } +#[derive(Debug)] pub struct FlatGroupIndexOperations; impl GroupIndexOperations for FlatGroupIndexOperations { 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 62ff887d446f..608d713bb75f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,11 +19,19 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator +use std::collections::VecDeque; +use std::fmt::{self, Debug}; +use std::marker::PhantomData; + use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; + +use crate::aggregate::groups_accumulator::{ + BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, +}; /// Track the accumulator null state per row: if any values for that /// group were null and if any values have been seen at all for that group. /// @@ -50,7 +58,7 @@ use datafusion_expr_common::groups_accumulator::EmitTo; /// /// [`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[i]` is true, have seen at least one non null @@ -58,19 +66,17 @@ pub struct NullState { /// /// If `seen_values[i]` is false, have not seen any values that /// pass the filter yet for group `i` - seen_values: BooleanBufferBuilder, -} + seen_values: V, -impl Default for NullState { - fn default() -> Self { - Self::new() - } + /// phantom data for required type `` + _phantom: PhantomData, } -impl NullState { +impl NullState { pub fn new() -> Self { Self { - seen_values: BooleanBufferBuilder::new(0), + seen_values: V::default(), + _phantom: PhantomData {}, } } @@ -105,15 +111,19 @@ impl NullState { mut value_fn: F, ) where T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, + F: FnMut(u32, u64, T::Native) + Send, { // ensure the seen_values is big enough (start everything at // "not seen" valid) - let seen_values = - initialize_builder(&mut self.seen_values, total_num_groups, false); - accumulate(group_indices, values, opt_filter, |group_index, value| { - seen_values.set_bit(group_index, true); - value_fn(group_index, value); + self.seen_values.resize(total_num_groups, false); + + let seen_values = &mut self.seen_values; + 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); + seen_values.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, value); }); } @@ -135,26 +145,29 @@ 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()); // ensure the seen_values is big enough (start everything at // "not seen" valid) - let seen_values = - initialize_builder(&mut self.seen_values, total_num_groups, false); + self.seen_values.resize(total_num_groups, false); // These could be made more performant by iterating in chunks of 64 bits at a time + let seen_values = &mut self.seen_values; match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { // 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.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value) }, ) } @@ -165,10 +178,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.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); } }) } @@ -180,10 +196,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.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); } }) } @@ -194,11 +213,14 @@ 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 { if 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.set_bit(block_id, block_offset, true); + value_fn(block_id, block_offset, new_value); } } }) @@ -212,7 +234,55 @@ impl NullState { /// /// resets the internal state appropriately pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { - let nulls: BooleanBuffer = self.seen_values.finish(); + self.seen_values.emit(emit_to) + } +} + +/// Seen values +pub trait SeenValues: Default + Debug + Send { + fn resize(&mut self, total_num_groups: usize, default_value: bool); + + fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool); + + fn emit(&mut self, emit_to: EmitTo) -> NullBuffer; + + fn capacity(&self) -> usize; +} + +#[derive(Debug)] +pub struct FlatSeenValues { + /// Have we seen any non-filtered input values for `group_index`? + /// + /// If `seen_values[i]` is true, have seen at least one non null + /// value for group `i` + /// + /// If `seen_values[i]` is false, have not seen any values that + /// pass the filter yet for group `i` + builder: BooleanBufferBuilder, +} + +impl Default for FlatSeenValues { + fn default() -> Self { + Self { + builder: BooleanBufferBuilder::new(0), + } + } +} + +impl SeenValues for FlatSeenValues { + fn resize(&mut self, total_num_groups: usize, default_value: bool) { + if self.builder.len() < total_num_groups { + let new_groups = total_num_groups - self.builder.len(); + self.builder.append_n(new_groups, default_value); + } + } + + fn set_bit(&mut self, _block_id: u32, block_offset: u64, value: bool) { + self.builder.set_bit(block_offset as usize, value); + } + + fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { + let nulls = self.builder.finish(); let nulls = match emit_to { EmitTo::All => nulls, @@ -224,16 +294,123 @@ impl NullState { let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); // reset the existing seen buffer for seen in nulls.iter().skip(n) { - self.seen_values.append(seen); + self.builder.append(seen); } first_n_null } - EmitTo::NextBlock(_) => todo!(), + EmitTo::NextBlock(_) => { + unreachable!("not support block emit in flat seen values") + } + }; + NullBuffer::new(nulls) + } + + fn capacity(&self) -> usize { + self.builder.capacity() + } +} + +#[derive(Debug, Default)] +pub struct BlockedSeenValues { + /// Have we seen any non-filtered input values for `group_index`? + /// + /// If `seen_values[i]` is true, have seen at least one non null + /// value for group `i` + /// + /// If `seen_values[i]` is false, have not seen any values that + /// pass the filter yet for group `i` + blocked_builders: VecDeque, + + block_size: usize, +} + +impl SeenValues for BlockedSeenValues { + fn resize(&mut self, total_num_groups: usize, default_value: bool) { + let block_size = self.block_size; + let blocked_builder = &mut self.blocked_builders; + + // For resize, we need to: + // 1. Ensure the blks are enough first + // 2. and then ensure slots in blks are enough + let (mut cur_blk_idx, exist_slots) = if blocked_builder.len() > 0 { + let cur_blk_idx = blocked_builder.len() - 1; + let exist_slots = (blocked_builder.len() - 1) * block_size + + blocked_builder.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) }; + + // No new groups, don't need to expand, just return + if exist_slots >= total_num_groups { + return; + } + + // 1. Ensure blks are enough + let exist_blks = blocked_builder.len(); + let new_blks = ((total_num_groups + block_size - 1) / block_size) - exist_blks; + if new_blks > 0 { + for _ in 0..new_blks { + blocked_builder.push_back(BooleanBufferBuilder::new(block_size)); + } + } + + // 2. Ensure slots are enough + let mut new_slots = total_num_groups - exist_slots; + + // 2.1 Only fill current blk if it may be already enough + let cur_blk_rest_slots = block_size - blocked_builder[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + blocked_builder[cur_blk_idx].append_n(new_slots, default_value); + return; + } + + // 2.2 Fill current blk to full + blocked_builder[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); + new_slots -= cur_blk_rest_slots; + + // 2.3 Fill complete blks + let complete_blks = new_slots / block_size; + for _ in 0..complete_blks { + cur_blk_idx += 1; + blocked_builder[cur_blk_idx].append_n(block_size, default_value); + } + + // 2.4 Fill last blk if needed + let rest_slots = new_slots % block_size; + if rest_slots > 0 { + blocked_builder + .back_mut() + .unwrap() + .append_n(rest_slots, default_value); + } + } + + fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool) { + self.blocked_builders[block_id as usize].set_bit(block_offset as usize, value); + } + + fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { + assert!(matches!(emit_to, EmitTo::NextBlock(_))); + + let mut block = self.blocked_builders.pop_front().expect(""); + let nulls = block.finish(); + NullBuffer::new(nulls) } + + fn capacity(&self) -> usize { + self.blocked_builders + .iter() + .map(|builder| builder.capacity()) + .sum() + } } +pub type FlatNullState = NullState; +pub type BlockedNullState = NullState; + /// Invokes `value_fn(group_index, value)` for each non null, non /// filtered value of `value`, /// @@ -798,15 +975,15 @@ mod test { total_num_groups: usize, ) { let mut accumulated_values = vec![]; - let mut null_state = NullState::new(); + let mut null_state = FlatNullState::new(); null_state.accumulate( group_indices, values, opt_filter, total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); + |_, group_index, value| { + accumulated_values.push((group_index as usize, value)); }, ); @@ -842,7 +1019,7 @@ mod test { assert_eq!(accumulated_values, expected_values, "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - let seen_values = null_state.seen_values.finish_cloned(); + let seen_values = null_state.seen_values.builder.finish_cloned(); mock.validate_seen_values(&seen_values); // Validate the final buffer (one value per group) @@ -914,15 +1091,15 @@ mod test { total_num_groups: usize, ) { let mut accumulated_values = vec![]; - let mut null_state = NullState::new(); + let mut null_state = FlatNullState::new(); null_state.accumulate_boolean( group_indices, values, opt_filter, total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); + |_, group_index, value| { + accumulated_values.push((group_index as usize, value)); }, ); @@ -959,7 +1136,7 @@ mod test { assert_eq!(accumulated_values, expected_values, "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - let seen_values = null_state.seen_values.finish_cloned(); + let seen_values = null_state.seen_values.builder.finish_cloned(); mock.validate_seen_values(&seen_values); // Validate the final buffer (one value per group) 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 e661e0510d37..b619e7ee9715 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 @@ -23,7 +23,7 @@ use arrow::buffer::BooleanBuffer; use datafusion_common::Result; 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(), 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); 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 d7b30b3a5775..103203d72a8d 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,7 @@ use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -use super::accumulate::NullState; +use super::accumulate::FlatNullState; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -53,7 +53,7 @@ where starting_value: T::Native, /// Track nulls in the input / filters - null_state: NullState, + null_state: FlatNullState, /// Function that computes the primitive result prim_fn: F, @@ -68,7 +68,7 @@ where Self { values: vec![], data_type: data_type.clone(), - null_state: NullState::new(), + null_state: FlatNullState::new(), starting_value: T::default_value(), prim_fn, } @@ -105,8 +105,8 @@ where values, opt_filter, total_num_groups, - |group_index, new_value| { - let value = &mut self.values[group_index]; + |_, group_index, new_value| { + let value = &mut self.values[group_index as usize]; (self.prim_fn)(value, new_value); }, ); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index ba21a1fd0832..b1972e2d8696 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -40,7 +40,7 @@ use datafusion_expr::{ ReversedUDAF, Signature, }; -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, }; @@ -533,7 +533,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, @@ -555,7 +555,7 @@ where sum_data_type: sum_data_type.clone(), counts: vec![], sums: vec![], - null_state: NullState::new(), + null_state: FlatNullState::new(), avg_fn, } } @@ -584,7 +584,8 @@ where values, opt_filter, total_num_groups, - |group_index, new_value| { + |_, group_index, new_value| { + let group_index = group_index as usize; let sum = &mut self.sums[group_index]; *sum = sum.add_wrapping(new_value); @@ -667,8 +668,8 @@ where 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; }, ); @@ -679,8 +680,8 @@ where partial_sums, opt_filter, total_num_groups, - |group_index, new_value: ::Native| { - let sum = &mut self.sums[group_index]; + |_, group_index, new_value: ::Native| { + let sum = &mut self.sums[group_index as usize]; *sum = sum.add_wrapping(new_value); }, ); diff --git a/datafusion/physical-expr/src/aggregate.rs b/datafusion/physical-expr/src/aggregate.rs index 49912954ac81..66fa71107d14 100644 --- a/datafusion/physical-expr/src/aggregate.rs +++ b/datafusion/physical-expr/src/aggregate.rs @@ -18,10 +18,10 @@ pub(crate) mod groups_accumulator { #[allow(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::{ - accumulate::NullState, GroupsAccumulatorAdapter, + accumulate::FlatNullState, GroupsAccumulatorAdapter, }; } pub(crate) mod stats { diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 9f795c81fa48..7c9272d9b048 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -47,7 +47,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::{GroupsAccumulatorAdapter, FlatNullState}; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ calculate_union, AcrossPartitions, ConstExpr, EquivalenceProperties, From 2dba944d15b1dfae4953dee0a03ee133bfefa5ad Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 13 Apr 2025 21:58:23 +0800 Subject: [PATCH 06/55] improve comments. --- .../groups_accumulator/accumulate.rs | 51 +++++++++++++------ 1 file changed, 36 insertions(+), 15 deletions(-) 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 608d713bb75f..ab2517fcb509 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -32,6 +32,7 @@ use datafusion_expr_common::groups_accumulator::EmitTo; use crate::aggregate::groups_accumulator::{ BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, }; + /// Track the accumulator null state per row: if any values for that /// group were null and if any values have been seen at all for that group. /// @@ -238,7 +239,7 @@ impl NullState { } } -/// Seen values +/// Structure marking if accumulating groups are seen at least one pub trait SeenValues: Default + Debug + Send { fn resize(&mut self, total_num_groups: usize, default_value: bool); @@ -249,15 +250,26 @@ pub trait SeenValues: Default + Debug + Send { fn capacity(&self) -> usize; } +/// [`SeenValues`] for `flat groups input` +/// +/// 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`. +/// #[derive(Debug)] pub struct FlatSeenValues { - /// Have we seen any non-filtered input values for `group_index`? - /// - /// If `seen_values[i]` is true, have seen at least one non null - /// value for group `i` - /// - /// If `seen_values[i]` is false, have not seen any values that - /// pass the filter yet for group `i` builder: BooleanBufferBuilder, } @@ -310,15 +322,24 @@ impl SeenValues for FlatSeenValues { } } +/// [`SeenValues`] for `blocked groups input` +/// +/// 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`. +/// #[derive(Debug, Default)] pub struct BlockedSeenValues { - /// Have we seen any non-filtered input values for `group_index`? - /// - /// If `seen_values[i]` is true, have seen at least one non null - /// value for group `i` - /// - /// If `seen_values[i]` is false, have not seen any values that - /// pass the filter yet for group `i` blocked_builders: VecDeque, block_size: usize, From c08f23af3ad65532424f930dfcff20e67cdf8e35 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 13 Apr 2025 22:06:04 +0800 Subject: [PATCH 07/55] remove stale codes. --- .../aggregate/groups_accumulator/accumulate.rs | 16 ---------------- 1 file changed, 16 deletions(-) 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 ab2517fcb509..fd347f5745fa 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -776,22 +776,6 @@ pub fn accumulate_indices( } } -/// Ensures that `builder` contains a `BooleanBufferBuilder with at -/// least `total_num_groups`. -/// -/// All new entries are initialized to `default_value` -fn initialize_builder( - builder: &mut BooleanBufferBuilder, - total_num_groups: usize, - default_value: bool, -) -> &mut BooleanBufferBuilder { - if builder.len() < total_num_groups { - let new_groups = total_num_groups - builder.len(); - builder.append_n(new_groups, default_value); - } - builder -} - #[cfg(test)] mod test { use super::*; From f6f3bd6c8bd8c35bfc94eddb531fe6422386f7cc Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 13 Apr 2025 23:20:11 +0800 Subject: [PATCH 08/55] add tests. --- .../groups_accumulator/accumulate.rs | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) 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 fd347f5745fa..066db1dface1 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -237,10 +237,66 @@ impl NullState { pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { self.seen_values.emit(emit_to) } + + /// Clone and build a single [`BooleanBuffer`] from `seen_values`, + /// only used for testing. + #[cfg(test)] + fn build_cloned_seen_values(&self) -> BooleanBuffer { + if let Some(seen_values) = + self.seen_values.as_any().downcast_ref::() + { + seen_values.builder.finish_cloned() + } else if let Some(seen_values) = self + .seen_values + .as_any() + .downcast_ref::() + { + let mut return_builder = BooleanBufferBuilder::new(0); + for builder in &seen_values.blocked_builders { + for idx in 0..builder.len() { + return_builder.append(builder.get_bit(idx)); + } + } + return_builder.finish() + } else { + unreachable!("unknown impl of SeenValues") + } + } + + /// Emit a single [`NullBuffer`], only used for testing. + #[cfg(test)] + fn emit_all_in_once(&mut self, total_num_groups: usize) -> NullBuffer { + if let Some(seen_values) = + self.seen_values.as_any().downcast_ref::() + { + seen_values.emit(EmitTo::All) + } else if let Some(seen_values) = self + .seen_values + .as_any() + .downcast_ref::() + { + let mut return_builder = BooleanBufferBuilder::new(0); + let num_blocks = seen_values.blocked_builders.len(); + for _ in 0..num_blocks { + let blocked_nulls = seen_values.emit(EmitTo::NextBlock(true)); + for bit in blocked_nulls.inner().iter() { + return_builder.append(bit); + } + } + + NullBuffer::new(return_builder.finish()) + } else { + unreachable!("unknown impl of SeenValues") + } + } } /// Structure marking if accumulating groups are seen at least one pub trait SeenValues: Default + Debug + Send { + fn as_any(&self) -> &dyn std::any::Any { + self + } + fn resize(&mut self, total_num_groups: usize, default_value: bool); fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool); @@ -585,6 +641,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], @@ -648,6 +706,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>, @@ -839,6 +899,9 @@ mod test { /// filter (defaults to None) filter: BooleanArray, + + /// + block_size: Option, } impl Fixture { From 53e8c8e0d9ff6f639d92d6bcd6a541be7001e613 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 16 Apr 2025 14:14:15 +0800 Subject: [PATCH 09/55] support dynamic dispatching for `NullState`. --- .../groups_accumulator/accumulate.rs | 270 +++++++++++++----- 1 file changed, 199 insertions(+), 71 deletions(-) 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 066db1dface1..b3f8ec30fe73 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -74,13 +74,6 @@ pub struct NullState { } impl NullState { - pub fn new() -> Self { - Self { - seen_values: V::default(), - _phantom: PhantomData {}, - } - } - /// return the size of all buffers allocated by this null state, not including self pub fn size(&self) -> usize { // capacity is in bits, so convert to bytes @@ -237,66 +230,10 @@ impl NullState { pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { self.seen_values.emit(emit_to) } - - /// Clone and build a single [`BooleanBuffer`] from `seen_values`, - /// only used for testing. - #[cfg(test)] - fn build_cloned_seen_values(&self) -> BooleanBuffer { - if let Some(seen_values) = - self.seen_values.as_any().downcast_ref::() - { - seen_values.builder.finish_cloned() - } else if let Some(seen_values) = self - .seen_values - .as_any() - .downcast_ref::() - { - let mut return_builder = BooleanBufferBuilder::new(0); - for builder in &seen_values.blocked_builders { - for idx in 0..builder.len() { - return_builder.append(builder.get_bit(idx)); - } - } - return_builder.finish() - } else { - unreachable!("unknown impl of SeenValues") - } - } - - /// Emit a single [`NullBuffer`], only used for testing. - #[cfg(test)] - fn emit_all_in_once(&mut self, total_num_groups: usize) -> NullBuffer { - if let Some(seen_values) = - self.seen_values.as_any().downcast_ref::() - { - seen_values.emit(EmitTo::All) - } else if let Some(seen_values) = self - .seen_values - .as_any() - .downcast_ref::() - { - let mut return_builder = BooleanBufferBuilder::new(0); - let num_blocks = seen_values.blocked_builders.len(); - for _ in 0..num_blocks { - let blocked_nulls = seen_values.emit(EmitTo::NextBlock(true)); - for bit in blocked_nulls.inner().iter() { - return_builder.append(bit); - } - } - - NullBuffer::new(return_builder.finish()) - } else { - unreachable!("unknown impl of SeenValues") - } - } } /// Structure marking if accumulating groups are seen at least one pub trait SeenValues: Default + Debug + Send { - fn as_any(&self) -> &dyn std::any::Any { - self - } - fn resize(&mut self, total_num_groups: usize, default_value: bool); fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool); @@ -401,6 +338,15 @@ pub struct BlockedSeenValues { block_size: usize, } +impl BlockedSeenValues { + pub fn new(block_size: usize) -> Self { + Self { + blocked_builders: VecDeque::new(), + block_size, + } + } +} + impl SeenValues for BlockedSeenValues { fn resize(&mut self, total_num_groups: usize, default_value: bool) { let block_size = self.block_size; @@ -471,7 +417,10 @@ impl SeenValues for BlockedSeenValues { fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { assert!(matches!(emit_to, EmitTo::NextBlock(_))); - let mut block = self.blocked_builders.pop_front().expect(""); + let mut block = self + .blocked_builders + .pop_front() + .expect("should not try to emit empty blocks"); let nulls = block.finish(); NullBuffer::new(nulls) @@ -485,9 +434,148 @@ impl SeenValues for BlockedSeenValues { } } +/// Adapter for supporting dynamic dispatching of [`FlatNullState`] and [`BlockedNullState`]. +/// For performance, the cost of batch-level dynamic dispatching is acceptable. +pub enum NullStateAdapter { + Flat(FlatNullState), + Blocked(BlockedNullState), +} + +impl NullStateAdapter { + pub fn new(block_size: Option) -> Self { + if let Some(blk_size) = block_size { + Self::Blocked(BlockedNullState::new(blk_size)) + } else { + Self::Flat(FlatNullState::new()) + } + } + + 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) -> NullBuffer { + match self { + NullStateAdapter::Flat(null_state) => null_state.build(emit_to), + NullStateAdapter::Blocked(null_state) => null_state.build(emit_to), + } + } + + /// Clone and build a single [`BooleanBuffer`] from `seen_values`, + /// only used for testing. + #[cfg(test)] + fn build_cloned_seen_values(&self) -> BooleanBuffer { + match self { + NullStateAdapter::Flat(null_state) => { + null_state.seen_values.builder.finish_cloned() + } + NullStateAdapter::Blocked(null_state) => { + let mut return_builder = BooleanBufferBuilder::new(0); + for builder in &null_state.seen_values.blocked_builders { + for idx in 0..builder.len() { + return_builder.append(builder.get_bit(idx)); + } + } + return_builder.finish() + } + } + } + + #[cfg(test)] + fn build_all_in_once(&mut self) -> NullBuffer { + 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.seen_values.blocked_builders.len(); + for _ in 0..num_blocks { + let blocked_nulls = null_state.build(EmitTo::NextBlock(true)); + for bit in blocked_nulls.inner().iter() { + return_builder.append(bit); + } + } + + NullBuffer::new(return_builder.finish()) + } + } + } +} + pub type FlatNullState = NullState; + +impl FlatNullState { + pub fn new() -> Self { + Self { + seen_values: FlatSeenValues::default(), + _phantom: PhantomData {}, + } + } +} + pub type BlockedNullState = NullState; +impl BlockedNullState { + pub fn new(block_size: usize) -> Self { + Self { + seen_values: BlockedSeenValues::new(block_size), + _phantom: PhantomData {}, + } + } +} + /// Invokes `value_fn(group_index, value)` for each non null, non /// filtered value of `value`, /// @@ -873,6 +961,7 @@ mod test { values, values_with_nulls, filter, + block_size: None, } .run() } @@ -953,6 +1042,7 @@ mod test { values, values_with_nulls, filter, + block_size: None, } } @@ -977,7 +1067,13 @@ 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, + ); // nulls, no filters Self::accumulate_test( @@ -985,6 +1081,7 @@ mod test { &values_with_nulls_array, None, total_num_groups, + self.block_size, ); // no nulls, filters @@ -993,6 +1090,7 @@ mod test { &values_array, Some(filter), total_num_groups, + self.block_size, ); // nulls, filters @@ -1001,6 +1099,7 @@ mod test { &values_with_nulls_array, Some(filter), total_num_groups, + self.block_size, ); } @@ -1012,12 +1111,14 @@ mod test { values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + block_size: Option, ) { Self::accumulate_values_test( group_indices, values, opt_filter, total_num_groups, + block_size, ); Self::accumulate_indices_test(group_indices, values.nulls(), opt_filter); @@ -1041,17 +1142,44 @@ mod test { values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + block_size: Option, ) { let mut accumulated_values = vec![]; - let mut null_state = FlatNullState::new(); + let (mut null_state, block_size, acc_group_indices) = if let Some(blk_size) = + block_size + { + let acc_group_indices = group_indices + .iter() + .copied() + .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::>(); + ( + NullStateAdapter::new(Some(blk_size)), + blk_size, + acc_group_indices, + ) + } else { + ( + NullStateAdapter::new(None), + 0, + group_indices.iter().copied().collect(), + ) + }; null_state.accumulate( - group_indices, + &acc_group_indices, values, opt_filter, total_num_groups, - |_, group_index, value| { - accumulated_values.push((group_index as usize, value)); + |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 as usize, value)); }, ); @@ -1087,13 +1215,13 @@ mod test { assert_eq!(accumulated_values, expected_values, "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - let seen_values = null_state.seen_values.builder.finish_cloned(); + let seen_values = null_state.build_cloned_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_all_in_once(); assert_eq!(null_buffer, expected_null_buffer); } From b5d231e02563387aa84cbf7d0a96d5a4bcfc39eb Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 16 Apr 2025 15:08:03 +0800 Subject: [PATCH 10/55] improve tests to cover `BlockedNullState`. --- .../src/aggregate/groups_accumulator.rs | 34 -- .../groups_accumulator/accumulate.rs | 345 +++++++++++++++--- 2 files changed, 288 insertions(+), 91 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index f7aa27826334..fc0815744522 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -554,37 +554,3 @@ impl GroupIndexOperations for FlatGroupIndexOperations { packed_index } } - -// #[cfg(test)] -// mod tests { -// use super::*; - -// #[test] -// fn test_blocked_group_index_build() { -// let group_index1 = 1; -// let group_index2 = (42_u64 << 32) | 2; -// let group_index3 = ((u32::MAX as u64) << 32) | 3; - -// let index_builder = BlockedGroupIndexBuilder::new(false); -// let flat1 = index_builder.build(group_index1 as usize); -// let flat2 = index_builder.build(group_index2 as usize); -// let flat3 = index_builder.build(group_index3 as usize); -// let expected1 = BlockedGroupIndex::new_from_parts(0, group_index1); -// let expected2 = BlockedGroupIndex::new_from_parts(0, group_index2); -// let expected3 = BlockedGroupIndex::new_from_parts(0, group_index3); -// assert_eq!(flat1, expected1); -// assert_eq!(flat2, expected2); -// assert_eq!(flat3, expected3); - -// let index_builder = BlockedGroupIndexBuilder::new(true); -// let blocked1 = index_builder.build(group_index1 as usize); -// let blocked2 = index_builder.build(group_index2 as usize); -// let blocked3 = index_builder.build(group_index3 as usize); -// let expected1 = BlockedGroupIndex::new_from_parts(0, 1); -// let expected2 = BlockedGroupIndex::new_from_parts(42, 2); -// let expected3 = BlockedGroupIndex::new_from_parts(u32::MAX, 3); -// assert_eq!(blocked1, expected1); -// assert_eq!(blocked2, expected2); -// assert_eq!(blocked3, expected3); -// } -// } 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 b3f8ec30fe73..41989bd469d4 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -930,7 +930,7 @@ mod test { use arrow::array::{Int32Array, UInt32Array}; use rand::{rngs::ThreadRng, Rng}; - use std::collections::HashSet; + use std::{cmp, collections::HashSet}; #[test] fn accumulate() { @@ -956,12 +956,14 @@ mod test { }) .collect(); + // Test flat style Fixture { group_indices, values, values_with_nulls, filter, - block_size: None, + block_size: 3, + acc_rounds: 5, } .run() } @@ -989,8 +991,10 @@ mod test { /// filter (defaults to None) filter: BooleanArray, - /// - block_size: Option, + /// block size for testing [`BlockedNullState`] + block_size: usize, + + acc_rounds: usize, } impl Fixture { @@ -1007,6 +1011,12 @@ mod test { let values: Vec = (0..num_values).map(|_| rng.gen()).collect(); + // random block size + let block_size = rng.gen_range(1..num_groups); + + // random acc rounds + let acc_rounds = rng.gen_range(1..=group_indices.len()); + // 10% chance of false // 10% change of null // 80% chance of true @@ -1042,7 +1052,8 @@ mod test { values, values_with_nulls, filter, - block_size: None, + block_size, + acc_rounds, } } @@ -1073,6 +1084,7 @@ mod test { None, total_num_groups, self.block_size, + self.acc_rounds, ); // nulls, no filters @@ -1082,6 +1094,7 @@ mod test { None, total_num_groups, self.block_size, + self.acc_rounds, ); // no nulls, filters @@ -1091,6 +1104,7 @@ mod test { Some(filter), total_num_groups, self.block_size, + self.acc_rounds, ); // nulls, filters @@ -1100,6 +1114,7 @@ mod test { Some(filter), total_num_groups, self.block_size, + self.acc_rounds, ); } @@ -1111,28 +1126,97 @@ mod test { values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - block_size: Option, + 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, - block_size, + 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 @@ -1143,45 +1227,105 @@ mod test { opt_filter: Option<&BooleanArray>, total_num_groups: usize, block_size: Option, + acc_rounds: Option, ) { + // 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) + .into_iter() + .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()] + }; + + (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![]; - let (mut null_state, block_size, acc_group_indices) = if let Some(blk_size) = - block_size + 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) { - let acc_group_indices = group_indices - .iter() - .copied() - .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::>(); - ( - NullStateAdapter::new(Some(blk_size)), - blk_size, - acc_group_indices, - ) - } else { - ( - NullStateAdapter::new(None), - 0, - group_indices.iter().copied().collect(), - ) - }; - - null_state.accumulate( - &acc_group_indices, - values, - opt_filter, - 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 as usize, value)); - }, - ); + 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 as usize, value)); + }, + ); + } // Figure out the expected values let mut expected_values = vec![]; @@ -1285,19 +1429,106 @@ 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 = FlatNullState::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) + .into_iter() + .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 as usize, 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 as usize, value)); + }, + ); + } // Figure out the expected values let mut expected_values = vec![]; @@ -1332,13 +1563,13 @@ mod test { assert_eq!(accumulated_values, expected_values, "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - let seen_values = null_state.seen_values.builder.finish_cloned(); + let seen_values = null_state.build_cloned_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_all_in_once(); assert_eq!(null_buffer, expected_null_buffer); } From a644b171f8b0ebd5f02874a5457e37eb6fd0b897 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 17 Apr 2025 20:15:08 +0800 Subject: [PATCH 11/55] complete the impl of blocked `GroupValuesPrimitive`. --- .../group_values/single_group_by/primitive.rs | 116 ++++++++++-------- 1 file changed, 66 insertions(+), 50 deletions(-) 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 2bfc3f446a1d..10665bb44546 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 @@ -156,7 +156,6 @@ where } fn size(&self) -> usize { - todo!() // self.map.capacity() * size_of::() + self.values.len } @@ -169,55 +168,72 @@ where } fn emit(&mut self, emit_to: EmitTo) -> Result> { - todo!() - // fn build_primitive( - // values: Vec, - // null_idx: Option, - // ) -> PrimitiveArray { - // let nulls = null_idx.map(|null_idx| { - // let mut buffer = NullBufferBuilder::new(values.len()); - // buffer.append_n_non_nulls(null_idx); - // buffer.append_null(); - // buffer.append_n_non_nulls(values.len() - null_idx - 1); - // // NOTE: The inner builder must be constructed as there is at least one null - // buffer.finish().unwrap() - // }); - // PrimitiveArray::::new(values.into(), nulls) - // } - - // let array: PrimitiveArray = match emit_to { - // EmitTo::All => { - // self.map.clear(); - // build_primitive(std::mem::take(&mut self.values), self.null_group.take()) - // } - // EmitTo::First(n) => { - // self.map.retain(|group_idx| { - // // Decrement group index by n - // match group_idx.checked_sub(n) { - // // Group index was >= n, shift value down - // Some(sub) => { - // *group_idx = sub; - // true - // } - // // Group index was < n, so remove from table - // None => false, - // } - // }); - // let null_group = match &mut self.null_group { - // Some(v) if *v >= n => { - // *v -= n; - // None - // } - // Some(_) => self.null_group.take(), - // None => None, - // }; - // let mut split = self.values.split_off(n); - // std::mem::swap(&mut self.values, &mut split); - // build_primitive(split, null_group) - // } - // }; - - // Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) + fn build_primitive( + values: Vec, + null_idx: Option, + ) -> PrimitiveArray { + let nulls = null_idx.map(|null_idx| { + let mut buffer = NullBufferBuilder::new(values.len()); + buffer.append_n_non_nulls(null_idx); + buffer.append_null(); + buffer.append_n_non_nulls(values.len() - null_idx - 1); + // NOTE: The inner builder must be constructed as there is at least one null + buffer.finish().unwrap() + }); + PrimitiveArray::::new(values.into(), nulls) + } + + let array: PrimitiveArray = match emit_to { + EmitTo::All => { + assert!( + self.block_size.is_none(), + "only support EmitTo::All in flat group values" + ); + + self.map.clear(); + build_primitive( + std::mem::take(self.values.back_mut().unwrap()), + self.null_group.take(), + ) + } + EmitTo::First(n) => { + assert!( + self.block_size.is_none(), + "only support EmitTo::First in flat group values" + ); + + self.map.retain(|group_idx| { + // Decrement group index by n + match group_idx.checked_sub(n) { + // Group index was >= n, shift value down + Some(sub) => { + *group_idx = sub; + true + } + // Group index was < n, so remove from table + None => false, + } + }); + let null_group = match &mut self.null_group { + Some(v) if *v >= n => { + *v -= n; + None + } + Some(_) => self.null_group.take(), + None => None, + }; + + let single_block = self.values.back_mut().unwrap(); + let mut split = single_block.split_off(n); + std::mem::swap(single_block, &mut split); + build_primitive(split, null_group) + } + EmitTo::NextBlock(_) => { + + } + }; + + Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) } fn clear_shrink(&mut self, batch: &RecordBatch) { From 67f03fc0d71288f36d3d6a0c250f87dbad0c40e0 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 20 Apr 2025 15:13:00 +0800 Subject: [PATCH 12/55] support blocked mode for `GroupValuesPrimitive`. --- .../expr-common/src/groups_accumulator.rs | 8 +- .../group_values/single_group_by/primitive.rs | 160 +++++++++++++++--- 2 files changed, 141 insertions(+), 27 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index a06ae90f5076..30e7905cb27c 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -25,7 +25,7 @@ use datafusion_common::{not_impl_err, DataFusionError, Result}; /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy)] 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`. @@ -35,11 +35,7 @@ pub enum EmitTo { First(usize), /// Emit next block in the blocked managed groups /// - /// The flag's meaning: - /// - `true` represents new groups still will be added, - /// and we need to shift the values down. - /// - `false` represents no new groups will be added again, - /// and we don't need to shift the values down. + /// Similar as `Emit::All`, will also clear all existing group indexes NextBlock(bool), } 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 10665bb44546..bb2ef3372bcc 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 @@ -136,7 +136,9 @@ where 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.back().unwrap().len() == block_size { + 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); } @@ -156,15 +158,20 @@ where } fn size(&self) -> usize { - // self.map.capacity() * size_of::() + self.values.len + self.map.capacity() * size_of::() + + self + .values + .iter() + .map(|blk| blk.len() * blk.allocated_size()) + .sum::() } fn is_empty(&self) -> bool { - self.values.is_empty() + self.len() == 0 } fn len(&self) -> usize { - self.values.len() + self.map.len() + self.null_group.map(|_| 1).unwrap_or_default() } fn emit(&mut self, emit_to: EmitTo) -> Result> { @@ -184,24 +191,29 @@ where } let array: PrimitiveArray = match emit_to { + // =============================================== + // Emitting in flat mode + // =============================================== EmitTo::All => { assert!( self.block_size.is_none(), - "only support EmitTo::All in flat group values" + "only support EmitTo::All in flat mode" ); self.map.clear(); build_primitive( std::mem::take(self.values.back_mut().unwrap()), - self.null_group.take(), + self.null_group.take().map(|idx| idx as usize), ) } + EmitTo::First(n) => { assert!( self.block_size.is_none(), - "only support EmitTo::First in flat group values" + "only support EmitTo::First in flat mode" ); + let n = n as u64; self.map.retain(|group_idx| { // Decrement group index by n match group_idx.checked_sub(n) { @@ -214,6 +226,7 @@ where None => false, } }); + let null_group = match &mut self.null_group { Some(v) if *v >= n => { *v -= n; @@ -224,12 +237,107 @@ where }; let single_block = self.values.back_mut().unwrap(); - let mut split = single_block.split_off(n); + let mut split = single_block.split_off(n as usize); std::mem::swap(single_block, &mut split); - build_primitive(split, null_group) + build_primitive(split, null_group.map(|idx| idx as usize)) + } + + // =============================================== + // Emitting in blocked mode + // =============================================== + // TODO: we should consider if it is necessary to support indices modifying + // in `EmitTo::NextBlock`. It is only used in spilling case, maybe we can + // always emit all in blocked mode. So, we just need to clear the map rather + // than doing expansive modification for each buck in it. + EmitTo::NextBlock(true) => { + assert!( + self.block_size.is_some(), + "only support EmitTo::Next in blocked group values" + ); + + // We only emit the first block(`block_id == 0`), + // so erase the entries with `block_id == 0`, and decrease entries with `block_id > 0` + self.map.retain(|packed_idx| { + let old_blk_id = + BlockedGroupIndexOperations::get_block_id(*packed_idx); + match old_blk_id.checked_sub(1) { + // `block_id > 0`, shift `block_id` down + Some(new_blk_id) => { + let blk_offset = + BlockedGroupIndexOperations::get_block_offset( + *packed_idx, + ); + let new_packed_idx = BlockedGroupIndexOperations::pack_index( + new_blk_id as u32, + blk_offset, + ); + *packed_idx = new_packed_idx; + + true + } + + // `block_id == 0`, so remove from table + None => false, + } + }); + + // Similar as `non-nulls`, if `block_id > 0` we decrease, and if `block_id == 0` we erase + let null_block_pair_opt = self.null_group.map(|packed_idx| { + ( + BlockedGroupIndexOperations::get_block_id(packed_idx), + BlockedGroupIndexOperations::get_block_offset(packed_idx), + ) + }); + 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_idx = BlockedGroupIndexOperations::pack_index( + new_blk_id, blk_offset, + ); + self.null_group = Some(new_packed_idx); + 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) } - EmitTo::NextBlock(_) => { - + + EmitTo::NextBlock(false) => { + assert!( + self.block_size.is_some(), + "only support EmitTo::Next in blocked group values" + ); + + let null_block_pair_opt = self.null_group.map(|packed_idx| { + ( + BlockedGroupIndexOperations::get_block_id(packed_idx), + BlockedGroupIndexOperations::get_block_offset(packed_idx), + ) + }); + 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_idx = BlockedGroupIndexOperations::pack_index( + new_blk_id, blk_offset, + ); + self.null_group = Some(new_packed_idx); + 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) } }; @@ -238,8 +346,16 @@ where fn clear_shrink(&mut self, batch: &RecordBatch) { let count = batch.num_rows(); - self.values.clear(); - self.values.shrink_to(count); + + // 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(count); + } + self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared } @@ -265,16 +381,17 @@ where for v in cols[0].as_primitive::() { let group_index = match v { None => *self.null_group.get_or_insert_with(|| { - // actions before add new group like checking if room is enough + // Actions before add new group like checking if room is enough before_add_group(&mut self.values); - // get block infos and update block - let block_id = self.values.len() as u32; + // 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(Default::default()); - // get group index and finish actions needed it + // Get group index and finish actions needed it O::pack_index(block_id, block_offset) }), Some(key) => { @@ -305,16 +422,17 @@ where match insert { hashbrown::hash_table::Entry::Occupied(o) => *o.get(), hashbrown::hash_table::Entry::Vacant(v) => { - // actions before add new group like checking if room is enough + // Actions before add new group like checking if room is enough before_add_group(&mut self.values); - // get block infos and update block - let block_id = self.values.len() as u32; + // 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 + // Get group index and finish actions needed it let packed_index = O::pack_index(block_id, block_offset); v.insert(packed_index); packed_index From d165fb0024583b872773034dd12dbb418ab61804 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 20 Apr 2025 15:24:56 +0800 Subject: [PATCH 13/55] make `EmitTo::NextBlock` simpler. --- .../expr-common/src/groups_accumulator.rs | 6 +- .../groups_accumulator/accumulate.rs | 6 +- .../aggregate/groups_accumulator/bool_op.rs | 2 +- .../functions-aggregate/src/correlation.rs | 4 +- .../functions-aggregate/src/first_last.rs | 4 +- .../src/min_max/min_max_bytes.rs | 2 +- .../group_values/multi_group_by/mod.rs | 2 +- .../src/aggregates/group_values/row.rs | 2 +- .../group_values/single_group_by/bytes.rs | 2 +- .../single_group_by/bytes_view.rs | 2 +- .../group_values/single_group_by/primitive.rs | 89 ++++++------------- 11 files changed, 42 insertions(+), 79 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 30e7905cb27c..042013eaf7f7 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -36,7 +36,7 @@ pub enum EmitTo { /// Emit next block in the blocked managed groups /// /// Similar as `Emit::All`, will also clear all existing group indexes - NextBlock(bool), + NextBlock, } impl EmitTo { @@ -75,7 +75,7 @@ impl EmitTo { std::mem::swap(v, &mut t); t } - Self::NextBlock(_) => unreachable!("don't support take block in take_needed"), + Self::NextBlock => unreachable!("don't support take block in take_needed"), } } @@ -85,7 +85,7 @@ impl EmitTo { /// fn take_needed_block(&self, blocks: &mut VecDeque>) -> Vec { assert!( - matches!(self, Self::NextBlock(_)), + matches!(self, Self::NextBlock), "only support take block in take_needed_block" ); blocks 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 41989bd469d4..a29d2a9b6028 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -303,7 +303,7 @@ impl SeenValues for FlatSeenValues { } first_n_null } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("not support block emit in flat seen values") } }; @@ -415,7 +415,7 @@ impl SeenValues for BlockedSeenValues { } fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { - assert!(matches!(emit_to, EmitTo::NextBlock(_))); + assert!(matches!(emit_to, EmitTo::NextBlock)); let mut block = self .blocked_builders @@ -542,7 +542,7 @@ impl NullStateAdapter { let mut return_builder = BooleanBufferBuilder::new(0); let num_blocks = null_state.seen_values.blocked_builders.len(); for _ in 0..num_blocks { - let blocked_nulls = null_state.build(EmitTo::NextBlock(true)); + let blocked_nulls = null_state.build(EmitTo::NextBlock); for bit in blocked_nulls.inner().iter() { return_builder.append(bit); } 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 b619e7ee9715..41c32d81b996 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 @@ -118,7 +118,7 @@ where } first_n } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this accumulator still not support blocked groups") } }; diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index 51902092cf25..191458e185a4 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -448,7 +448,7 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { let n = match emit_to { EmitTo::All => self.count.len(), EmitTo::First(n) => n, - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this accumulator still not support blocked groups") } }; @@ -504,7 +504,7 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { let n = match emit_to { EmitTo::All => self.count.len(), EmitTo::First(n) => n, - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this accumulator still not support blocked groups") } }; diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 5b4ae6566a31..c25da41225a1 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -436,7 +436,7 @@ where self.size_of_orderings -= result.iter().map(ScalarValue::size_of_vec).sum::() } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this accumulator still not support blocked groups") } } @@ -463,7 +463,7 @@ where } first_n } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this group values still 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 45871bbd20eb..3615a8b919de 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -511,7 +511,7 @@ impl MinMaxBytesState { self.total_data_bytes -= first_data_capacity; (first_data_capacity, first_min_maxes) } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this accumulator still not support blocked groups") } } 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 bb8bd765c2ca..fb5f256c52af 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 @@ -1154,7 +1154,7 @@ impl GroupValues for GroupValuesColumn { output } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this group values still not support blocked groups") } }; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 8022ac637d1b..548e5fe38e8f 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -229,7 +229,7 @@ impl GroupValues for GroupValuesRows { }); output } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this group values still not support blocked groups") } }; 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 e3768b78d9b3..bdc02bcc34e7 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 @@ -116,7 +116,7 @@ impl GroupValues for GroupValuesByes { emit_group_values } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this group values still not support blocked groups") } }; 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 551f4c2eb530..610fea924a05 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 @@ -117,7 +117,7 @@ impl GroupValues for GroupValuesBytesView { emit_group_values } - EmitTo::NextBlock(_) => { + EmitTo::NextBlock => { unreachable!("this group values still not support blocked groups") } }; 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 bb2ef3372bcc..e9ba3ffc8a18 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 @@ -115,6 +115,9 @@ pub struct GroupValuesPrimitive { 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()); @@ -245,74 +248,15 @@ where // =============================================== // Emitting in blocked mode // =============================================== - // TODO: we should consider if it is necessary to support indices modifying - // in `EmitTo::NextBlock`. It is only used in spilling case, maybe we can - // always emit all in blocked mode. So, we just need to clear the map rather - // than doing expansive modification for each buck in it. - EmitTo::NextBlock(true) => { + EmitTo::NextBlock => { assert!( self.block_size.is_some(), "only support EmitTo::Next in blocked group values" ); - // We only emit the first block(`block_id == 0`), - // so erase the entries with `block_id == 0`, and decrease entries with `block_id > 0` - self.map.retain(|packed_idx| { - let old_blk_id = - BlockedGroupIndexOperations::get_block_id(*packed_idx); - match old_blk_id.checked_sub(1) { - // `block_id > 0`, shift `block_id` down - Some(new_blk_id) => { - let blk_offset = - BlockedGroupIndexOperations::get_block_offset( - *packed_idx, - ); - let new_packed_idx = BlockedGroupIndexOperations::pack_index( - new_blk_id as u32, - blk_offset, - ); - *packed_idx = new_packed_idx; - - true - } - - // `block_id == 0`, so remove from table - None => false, - } - }); - - // Similar as `non-nulls`, if `block_id > 0` we decrease, and if `block_id == 0` we erase - let null_block_pair_opt = self.null_group.map(|packed_idx| { - ( - BlockedGroupIndexOperations::get_block_id(packed_idx), - BlockedGroupIndexOperations::get_block_offset(packed_idx), - ) - }); - 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_idx = BlockedGroupIndexOperations::pack_index( - new_blk_id, blk_offset, - ); - self.null_group = Some(new_packed_idx); - 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) - } - - EmitTo::NextBlock(false) => { - assert!( - self.block_size.is_some(), - "only support EmitTo::Next in blocked group values" - ); + // 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_idx| { ( @@ -359,6 +303,25 @@ where self.map.clear(); self.map.shrink_to(count, |_| 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 From 319e1355117475d26c747adb3167b9a0030a31da Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 20 Apr 2025 21:52:35 +0800 Subject: [PATCH 14/55] extract the common codes of block for reusing. --- .../src/aggregate/groups_accumulator.rs | 84 +++++++++++++++++++ .../groups_accumulator/accumulate.rs | 81 +++++------------- 2 files changed, 107 insertions(+), 58 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index fc0815744522..86ca22c368f6 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -23,6 +23,7 @@ pub mod bool_op; pub mod nulls; pub mod prim_op; +use std::collections::VecDeque; use std::fmt::Debug; use std::mem::{size_of, size_of_val}; @@ -509,6 +510,10 @@ pub(crate) fn slice_and_maybe_filter( } } +// =============================================== +// Useful tools for group index +// =============================================== + /// Blocked style group index used in blocked mode group values and accumulators /// - High 32 bits represent `block_id` /// - Low 32 bits represent `block_offset` @@ -554,3 +559,82 @@ impl GroupIndexOperations for FlatGroupIndexOperations { packed_index } } + +// =============================================== +// Useful tools for block +// =============================================== +pub(crate) fn ensure_room_enough_for_blocks( + blocks: &mut VecDeque, + total_num_groups: usize, + block_size: usize, + new_block: F, + default_value: B::T, +) where + B: Block, + F: Fn(usize) -> B, +{ + // For resize, we need to: + // 1. Ensure the blks are enough first + // 2. and then ensure slots in blks are enough + let (mut cur_blk_idx, exist_slots) = if blocks.len() > 0 { + let cur_blk_idx = blocks.len() - 1; + let exist_slots = (blocks.len() - 1) * block_size + blocks.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + + // No new groups, don't need to expand, just return + if exist_slots >= total_num_groups { + return; + } + + // 1. Ensure blks are enough + let exist_blks = blocks.len(); + let new_blks = ((total_num_groups + block_size - 1) / block_size) - exist_blks; + if new_blks > 0 { + for _ in 0..new_blks { + let block = new_block(block_size); + blocks.push_back(block); + } + } + + // 2. Ensure slots are enough + let mut new_slots = total_num_groups - exist_slots; + + // 2.1 Only fill current blk if it may be already enough + let cur_blk_rest_slots = block_size - blocks[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + blocks[cur_blk_idx].fill_default_value(new_slots, default_value.clone()); + return; + } + + // 2.2 Fill current blk to full + blocks[cur_blk_idx].fill_default_value(cur_blk_rest_slots, default_value.clone()); + new_slots -= cur_blk_rest_slots; + + // 2.3 Fill complete blks + let complete_blks = new_slots / block_size; + for _ in 0..complete_blks { + cur_blk_idx += 1; + blocks[cur_blk_idx].fill_default_value(block_size, default_value.clone()); + } + + // 2.4 Fill last blk if needed + let rest_slots = new_slots % block_size; + if rest_slots > 0 { + blocks + .back_mut() + .unwrap() + .fill_default_value(rest_slots, default_value); + } +} + +pub(crate) trait Block { + type T: Clone; + + fn len(&self) -> usize; + + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); +} 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 a29d2a9b6028..d06024c49140 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -30,7 +30,8 @@ use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; use crate::aggregate::groups_accumulator::{ - BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, + ensure_room_enough_for_blocks, Block, BlockedGroupIndexOperations, + FlatGroupIndexOperations, GroupIndexOperations, }; /// Track the accumulator null state per row: if any values for that @@ -347,67 +348,31 @@ impl BlockedSeenValues { } } -impl SeenValues for BlockedSeenValues { - fn resize(&mut self, total_num_groups: usize, default_value: bool) { - let block_size = self.block_size; - let blocked_builder = &mut self.blocked_builders; - - // For resize, we need to: - // 1. Ensure the blks are enough first - // 2. and then ensure slots in blks are enough - let (mut cur_blk_idx, exist_slots) = if blocked_builder.len() > 0 { - let cur_blk_idx = blocked_builder.len() - 1; - let exist_slots = (blocked_builder.len() - 1) * block_size - + blocked_builder.back().unwrap().len(); - - (cur_blk_idx, exist_slots) - } else { - (0, 0) - }; - - // No new groups, don't need to expand, just return - if exist_slots >= total_num_groups { - return; - } - - // 1. Ensure blks are enough - let exist_blks = blocked_builder.len(); - let new_blks = ((total_num_groups + block_size - 1) / block_size) - exist_blks; - if new_blks > 0 { - for _ in 0..new_blks { - blocked_builder.push_back(BooleanBufferBuilder::new(block_size)); - } - } +impl Block for BooleanBufferBuilder { + type T = bool; - // 2. Ensure slots are enough - let mut new_slots = total_num_groups - exist_slots; - - // 2.1 Only fill current blk if it may be already enough - let cur_blk_rest_slots = block_size - blocked_builder[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - blocked_builder[cur_blk_idx].append_n(new_slots, default_value); - return; - } + fn len(&self) -> usize { + self.len() + } - // 2.2 Fill current blk to full - blocked_builder[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); - new_slots -= cur_blk_rest_slots; + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { + self.append_n(fill_len, default_value); + } +} - // 2.3 Fill complete blks - let complete_blks = new_slots / block_size; - for _ in 0..complete_blks { - cur_blk_idx += 1; - blocked_builder[cur_blk_idx].append_n(block_size, default_value); - } +impl SeenValues for BlockedSeenValues { + fn resize(&mut self, total_num_groups: usize, default_value: bool) { + let block_size = self.block_size; + let blocked_builders = &mut self.blocked_builders; + let new_block = |block_size: usize| BooleanBufferBuilder::new(block_size); - // 2.4 Fill last blk if needed - let rest_slots = new_slots % block_size; - if rest_slots > 0 { - blocked_builder - .back_mut() - .unwrap() - .append_n(rest_slots, default_value); - } + ensure_room_enough_for_blocks( + blocked_builders, + total_num_groups, + block_size, + new_block, + default_value, + ); } fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool) { From 808f14291f5d3e144847534352db517ff182b472 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 20 Apr 2025 22:23:36 +0800 Subject: [PATCH 15/55] support blocked mode for `PrimitiveGroupsAccumulator`. --- .../groups_accumulator/accumulate.rs | 10 ++- .../aggregate/groups_accumulator/prim_op.rs | 82 ++++++++++++++++--- 2 files changed, 79 insertions(+), 13 deletions(-) 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 d06024c49140..6d78ef4e2555 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -20,7 +20,7 @@ //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator use std::collections::VecDeque; -use std::fmt::{self, Debug}; +use std::fmt::Debug; use std::marker::PhantomData; use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; @@ -401,6 +401,7 @@ impl SeenValues for BlockedSeenValues { /// 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), @@ -479,6 +480,13 @@ impl NullStateAdapter { } } + 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)] 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 103203d72a8d..003831277ef3 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 @@ -15,10 +15,12 @@ // specific language governing permissions and limitations // under the License. +use std::collections::VecDeque; +use std::iter; use std::mem::size_of; use std::sync::Arc; -use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; +use arrow::array::{ArrayRef, ArrowNativeTypeOp, AsArray, BooleanArray, PrimitiveArray}; use arrow::buffer::NullBuffer; use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; @@ -26,7 +28,8 @@ use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -use super::accumulate::FlatNullState; +use crate::aggregate::groups_accumulator::accumulate::NullStateAdapter; +use crate::aggregate::groups_accumulator::{ensure_room_enough_for_blocks, Block}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -43,8 +46,8 @@ where T: ArrowPrimitiveType + Send, F: Fn(&mut T::Native, T::Native) + Send + Sync, { - /// values per group, stored as the native type - values: Vec, + /// Values per group, stored as the native type + values: VecDeque>, /// The output type (needed for Decimal precision and scale) data_type: DataType, @@ -53,10 +56,20 @@ where starting_value: T::Native, /// Track nulls in the input / filters - null_state: FlatNullState, + null_state: NullStateAdapter, /// Function that computes the primitive result prim_fn: F, + + /// Block size of current `GroupAccumulator` 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 PrimitiveGroupsAccumulator @@ -66,11 +79,12 @@ where { pub fn new(data_type: &DataType, prim_fn: F) -> Self { Self { - values: vec![], + values: VecDeque::new(), data_type: data_type.clone(), - null_state: FlatNullState::new(), + null_state: NullStateAdapter::new(None), starting_value: T::default_value(), prim_fn, + block_size: None, } } @@ -96,8 +110,28 @@ 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 + if let Some(blk_size) = self.block_size { + // Expand blocks in `blocked mode` + let new_block = |block_size: usize| Vec::with_capacity(block_size); + ensure_room_enough_for_blocks( + &mut self.values, + total_num_groups, + blk_size, + new_block, + self.starting_value, + ); + } else { + // Expand the single block in `flat mode` + if self.values.is_empty() { + self.values.push_back(Vec::new()); + } + + self.values + .back_mut() + .unwrap() + .resize(total_num_groups, self.starting_value); + } // NullState dispatches / handles tracking nulls and groups that saw no values self.null_state.accumulate( @@ -105,8 +139,8 @@ where values, opt_filter, total_num_groups, - |_, group_index, new_value| { - let value = &mut self.values[group_index as usize]; + |block_id, block_offset, new_value| { + let value = &mut self.values[block_id as usize][block_offset as usize]; (self.prim_fn)(value, new_value); }, ); @@ -115,7 +149,7 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = emit_to.take_needed_rows(&mut self.values); + let values = emit_to.take_needed(&mut self.values, self.block_size.is_some()); let nulls = self.null_state.build(emit_to); let values = PrimitiveArray::::new(values.into(), Some(nulls)) // no copy .with_data_type(self.data_type.clone()); @@ -198,4 +232,28 @@ where fn size(&self) -> usize { self.values.capacity() * size_of::() + self.null_state.size() } + + fn supports_blocked_groups(&self) -> bool { + true + } + + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { + self.values.clear(); + self.null_state = NullStateAdapter::new(block_size); + self.block_size = block_size; + + Ok(()) + } +} + +impl Block for Vec { + type T = N; + + fn len(&self) -> usize { + self.len() + } + + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { + self.extend(iter::repeat(default_value.clone()).take(fill_len)); + } } From 489f093e3f5cbc0baa1299e92fda70232bcf2c76 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 21 Apr 2025 16:40:34 +0800 Subject: [PATCH 16/55] impl block based result returning logic. --- .../physical-plan/src/aggregates/row_hash.rs | 67 +++++++++++++++++-- 1 file changed, 61 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 232565a04466..c3cde5b09494 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -42,7 +42,7 @@ use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use datafusion_execution::TaskContext; +use datafusion_execution::{DiskManager, TaskContext}; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; @@ -62,6 +62,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. /// @@ -423,6 +428,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. @@ -615,6 +623,7 @@ impl GroupedHashAggregateStream { spill_state, group_values_soft_limit: agg.limit, skip_aggregation_probe, + enable_blocked_groups: false, }) } } @@ -802,6 +811,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(emit_to, false); + let Ok(batch_opt) = emit_result else { + return Poll::Ready(Some(emit_result)); + }; + + 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!(output_batch.num_rows() > 0); + return Poll::Ready(Some(Ok( + batch.record_output(&self.baseline_metrics) + ))); + } + ExecutionState::Done => { // release the memory reservation since sending back output batch itself needs // some memory reservation, so make some room for it. @@ -982,6 +1018,9 @@ impl GroupedHashAggregateStream { && self.update_memory_reservation().is_err() { assert_ne!(self.mode, AggregateMode::Partial); + // TODO: support spilling when blocked group optimization is on + // (`enable_blocked_groups` is true) + assert!(!self.enable_blocked_groups); self.spill()?; self.clear_shrink(batch); } @@ -1033,11 +1072,16 @@ impl GroupedHashAggregateStream { /// Currently only [`GroupOrdering::None`] is supported for early emitting. /// TODO: support group_ordering for early emitting fn emit_early_if_necessary(&mut self) -> Result<()> { + // TODO: support spilling when blocked group optimization is on + // (`enable_blocked_groups` is true) if self.group_values.len() >= self.batch_size && matches!(self.group_ordering, GroupOrdering::None) && self.update_memory_reservation().is_err() { assert_eq!(self.mode, AggregateMode::Partial); + // TODO: support spilling when blocked group optimization is on + // (`enable_blocked_groups` is true) + assert!(!self.enable_blocked_groups); let n = self.group_values.len() / self.batch_size * self.batch_size; if let Some(batch) = self.emit(EmitTo::First(n), false)? { self.exec_state = ExecutionState::ProducingOutput(batch); @@ -1100,9 +1144,16 @@ impl GroupedHashAggregateStream { let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); let timer = elapsed_compute.timer(); self.exec_state = if self.spill_state.spills.is_empty() { - let batch = self.emit(EmitTo::All, false)?; - batch.map_or(ExecutionState::Done, ExecutionState::ProducingOutput) + if !self.enable_blocked_groups { + let batch = self.emit(EmitTo::All, false)?; + batch.map_or(ExecutionState::Done, ExecutionState::ProducingOutput) + } else { + ExecutionState::ProducingBlocks + } } else { + // TODO: support spilling when blocked group optimization is on + // (`enable_blocked_groups` is true) + assert!(!self.enable_blocked_groups); // If spill files exist, stream-merge them. self.update_merged_stream()?; ExecutionState::ReadingInput @@ -1130,9 +1181,13 @@ impl GroupedHashAggregateStream { fn switch_to_skip_aggregation(&mut self) -> Result<()> { if let Some(probe) = self.skip_aggregation_probe.as_mut() { if probe.should_skip() { - if let Some(batch) = self.emit(EmitTo::All, false)? { - self.exec_state = ExecutionState::ProducingOutput(batch); - }; + if !self.enable_blocked_groups { + if let Some(batch) = self.emit(EmitTo::All, false)? { + self.exec_state = ExecutionState::ProducingOutput(batch); + }; + } else { + self.exec_state = ExecutionState::ProducingBlocks; + } } } From 54002a14ee2b9e66d4ad59e2ee84fe553a7e1d36 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 21 Apr 2025 20:44:33 +0800 Subject: [PATCH 17/55] add judgement about when we should enable blocked groups optimization. --- datafusion/execution/src/runtime_env.rs | 18 +++++- .../physical-plan/src/aggregates/row_hash.rs | 58 ++++++++++++++++++- 2 files changed, 70 insertions(+), 6 deletions(-) diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index cb085108819e..82898968c0e0 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -253,12 +253,24 @@ impl RuntimeEnvBuilder { cache_manager, object_store_registry, } = self; - let memory_pool = - memory_pool.unwrap_or_else(|| Arc::new(UnboundedMemoryPool::default())); + + // If `memory_pool` is not set, it represents spilling is disabled. + // Because `UnboundedMemoryPool` will be used in this case, and + // it will never limit memory usage. + // And we disable the `disk_manager`(use `DiskManagerConfig::Disabled`) + // to tell we have disabled spilling. + let (memory_pool, disk_manager_config) = if let Some(pool) = memory_pool { + (pool, disk_manager) + } else { + ( + Arc::new(UnboundedMemoryPool::default()), + DiskManagerConfig::Disabled, + ) + }; Ok(RuntimeEnv { memory_pool, - disk_manager: DiskManager::try_new(disk_manager)?, + disk_manager: DiskManager::try_new(disk_manager_config)?, cache_manager: CacheManager::try_new(&cache_manager)?, object_store_registry, }) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index c3cde5b09494..3c313aa1435e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -486,7 +486,7 @@ impl GroupedHashAggregateStream { }; // Instantiate the accumulators - let accumulators: Vec<_> = aggregate_exprs + let mut accumulators: Vec<_> = aggregate_exprs .iter() .map(create_group_accumulator) .collect::>()?; @@ -551,7 +551,7 @@ impl GroupedHashAggregateStream { ordering.as_ref(), )?; - let group_values = new_group_values(group_schema, &group_ordering)?; + let mut group_values = new_group_values(group_schema, &group_ordering)?; timer.done(); let exec_state = ExecutionState::ReadingInput; @@ -604,6 +604,15 @@ impl GroupedHashAggregateStream { None }; + // Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. + let enable_blocked_groups = maybe_enable_blocked_groups( + &context, + group_values.as_mut(), + &mut accumulators, + batch_size, + &group_ordering, + )?; + Ok(GroupedHashAggregateStream { schema: agg_schema, input, @@ -623,7 +632,7 @@ impl GroupedHashAggregateStream { spill_state, group_values_soft_limit: agg.limit, skip_aggregation_probe, - enable_blocked_groups: false, + enable_blocked_groups, }) } } @@ -648,6 +657,49 @@ pub(crate) fn create_group_accumulator( } } +/// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. +/// The blocked optimization will be enabled when: +/// - When `enable_aggregation_intermediate_states_blocked_approach` is true +/// - 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) +/// - The accumulator is not empty(I am still not sure about logic in this case) +/// - [`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? +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 + if !matches!(group_ordering, GroupOrdering::None) + || accumulators.is_empty() + || context.runtime_env().disk_manager.tmp_files_enabled() + { + 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; From 1f7b4bc07ed34e91b634be431293ecd4961d1760 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 21 Apr 2025 21:26:06 +0800 Subject: [PATCH 18/55] add config to control if we enable blocked groups optimization. --- datafusion/common/src/config.rs | 12 ++++++++++++ datafusion/physical-plan/src/aggregates/row_hash.rs | 10 +++++++--- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 1e0f63d6d81c..3e68ddce0d06 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -405,6 +405,18 @@ config_namespace! { /// in joins can reduce memory usage when joining large /// tables with a highly-selective join filter, but is also slightly slower. pub enforce_batch_size_in_joins: bool, default = false + + /// Should DataFusion use the the blocked approach to manage the groups + /// values and their related states in accumulators. + /// By default, the blocked approach will be used. And the blocked approach + /// allocates capacity for the block 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 setting this flag to `false`, will fall-back to use the single approach, + /// values 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. + pub enable_aggregation_blocked_groups: bool, default = true } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 3c313aa1435e..e5d44314561b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -659,7 +659,7 @@ pub(crate) fn create_group_accumulator( /// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. /// The blocked optimization will be enabled when: -/// - When `enable_aggregation_intermediate_states_blocked_approach` is true +/// - When `enable_aggregation_blocked_groups` is true(default to true) /// - 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) /// - The accumulator is not empty(I am still not sure about logic in this case) @@ -676,8 +676,12 @@ fn maybe_enable_blocked_groups( block_size: usize, group_ordering: &GroupOrdering, ) -> Result { - // if !context.session_config().options().execution - if !matches!(group_ordering, GroupOrdering::None) + if !context + .session_config() + .options() + .execution + .enable_aggregation_blocked_groups + || !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() || context.runtime_env().disk_manager.tmp_files_enabled() { From e58afa5e84652aee81f50b86e0968aafd770d66d Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 21 Apr 2025 21:38:38 +0800 Subject: [PATCH 19/55] fix e2e sql tests. --- datafusion/execution/src/runtime_env.rs | 2 +- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 8 ++++---- datafusion/sqllogictest/test_files/information_schema.slt | 2 ++ 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index 82898968c0e0..ba5defb54d09 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -263,7 +263,7 @@ impl RuntimeEnvBuilder { (pool, disk_manager) } else { ( - Arc::new(UnboundedMemoryPool::default()), + Arc::new(UnboundedMemoryPool::default()) as _, DiskManagerConfig::Disabled, ) }; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 7c9272d9b048..f7e5f7c7d2ea 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -47,7 +47,7 @@ pub mod execution_props { pub use datafusion_expr::var_provider::{VarProvider, VarType}; } -pub use aggregate::groups_accumulator::{GroupsAccumulatorAdapter, FlatNullState}; +pub use aggregate::groups_accumulator::{FlatNullState, GroupsAccumulatorAdapter}; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ calculate_union, AcrossPartitions, ConstExpr, EquivalenceProperties, diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index e5d44314561b..942ff27a7890 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -42,7 +42,7 @@ use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; -use datafusion_execution::{DiskManager, TaskContext}; +use datafusion_execution::TaskContext; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{GroupsAccumulatorAdapter, PhysicalSortExpr}; @@ -872,9 +872,9 @@ impl Stream for GroupedHashAggregateStream { // - 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(emit_to, false); + let emit_result = self.emit(EmitTo::NextBlock, false); let Ok(batch_opt) = emit_result else { - return Poll::Ready(Some(emit_result)); + return Poll::Ready(Some(Err(emit_result.unwrap_err()))); }; let Some(batch) = batch_opt else { @@ -888,7 +888,7 @@ impl Stream for GroupedHashAggregateStream { continue; }; - debug_assert!(output_batch.num_rows() > 0); + debug_assert!(batch.num_rows() > 0); return Poll::Ready(Some(Ok( batch.record_output(&self.baseline_metrics) ))); diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 87abaadb516f..d55e63ea0e24 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 false +datafusion.execution.enable_aggregation_blocked_groups true datafusion.execution.enable_recursive_ctes true datafusion.execution.enforce_batch_size_in_joins false datafusion.execution.keep_partition_by_columns false @@ -317,6 +318,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 false Should DataFusion collect statistics after listing files +datafusion.execution.enable_aggregation_blocked_groups true Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the blocked approach will be used. And the blocked approach allocates capacity for the block 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 setting this flag to `false`, will fall-back to use the single approach, values 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. 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. datafusion.execution.keep_partition_by_columns false Should DataFusion keep the columns used for partition_by in the output RecordBatches From 4294ab7b3b59e73edb8a659bba7c1c02ca4b3224 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 21 Apr 2025 21:53:24 +0800 Subject: [PATCH 20/55] fix group values len method. --- .../src/aggregates/group_values/single_group_by/primitive.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e9ba3ffc8a18..252940fa753b 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 @@ -174,7 +174,7 @@ where } fn len(&self) -> usize { - self.map.len() + self.null_group.map(|_| 1).unwrap_or_default() + self.values.iter().map(|block| block.len()).sum::() } fn emit(&mut self, emit_to: EmitTo) -> Result> { From 79714a4d503f1104d7ce4b0868efc6711f234633 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 22 Apr 2025 14:24:02 +0800 Subject: [PATCH 21/55] add `memory_limit` to expose the info in `MemoryPool`. --- datafusion/execution/src/memory_pool/mod.rs | 15 +++++++++++++++ datafusion/execution/src/memory_pool/pool.rs | 18 +++++++++++++++++- datafusion/execution/src/runtime_env.rs | 18 +++--------------- 3 files changed, 35 insertions(+), 16 deletions(-) diff --git a/datafusion/execution/src/memory_pool/mod.rs b/datafusion/execution/src/memory_pool/mod.rs index 625a779b3eea..176303eaa1d6 100644 --- a/datafusion/execution/src/memory_pool/mod.rs +++ b/datafusion/execution/src/memory_pool/mod.rs @@ -141,6 +141,21 @@ pub trait MemoryPool: Send + Sync + std::fmt::Debug { /// Return the total amount of memory reserved fn reserved(&self) -> usize; + + /// Return the memory limit of the pool + /// + /// It is useful to know the limit of the pool sometime (e.g. we can know + /// if spilling will be triggered, it only we be triggered when the memory + /// limit isn't infinite) + fn memory_limit(&self) -> MemoryLimit { + MemoryLimit::Unknown + } +} + +pub enum MemoryLimit { + Infinite, + Finite(usize), + Unknown, } /// A memory consumer is a named allocation traced by a particular diff --git a/datafusion/execution/src/memory_pool/pool.rs b/datafusion/execution/src/memory_pool/pool.rs index cd6863939d27..e623246eb976 100644 --- a/datafusion/execution/src/memory_pool/pool.rs +++ b/datafusion/execution/src/memory_pool/pool.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; +use crate::memory_pool::{MemoryConsumer, MemoryLimit, MemoryPool, MemoryReservation}; use datafusion_common::HashMap; use datafusion_common::{resources_datafusion_err, DataFusionError, Result}; use log::debug; @@ -48,6 +48,10 @@ impl MemoryPool for UnboundedMemoryPool { fn reserved(&self) -> usize { self.used.load(Ordering::Relaxed) } + + fn memory_limit(&self) -> MemoryLimit { + MemoryLimit::Infinite + } } /// A [`MemoryPool`] that implements a greedy first-come first-serve limit. @@ -100,6 +104,10 @@ impl MemoryPool for GreedyMemoryPool { fn reserved(&self) -> usize { self.used.load(Ordering::Relaxed) } + + fn memory_limit(&self) -> MemoryLimit { + MemoryLimit::Finite(self.pool_size) + } } /// A [`MemoryPool`] that prevents spillable reservations from using more than @@ -233,6 +241,10 @@ impl MemoryPool for FairSpillPool { let state = self.state.lock(); state.spillable + state.unspillable } + + fn memory_limit(&self) -> MemoryLimit { + MemoryLimit::Finite(self.pool_size) + } } /// Constructs a resources error based upon the individual [`MemoryReservation`]. @@ -408,6 +420,10 @@ impl MemoryPool for TrackConsumersPool { fn reserved(&self) -> usize { self.inner.reserved() } + + fn memory_limit(&self) -> MemoryLimit { + self.inner.memory_limit() + } } fn provide_top_memory_consumers_to_error_msg( diff --git a/datafusion/execution/src/runtime_env.rs b/datafusion/execution/src/runtime_env.rs index ba5defb54d09..cb085108819e 100644 --- a/datafusion/execution/src/runtime_env.rs +++ b/datafusion/execution/src/runtime_env.rs @@ -253,24 +253,12 @@ impl RuntimeEnvBuilder { cache_manager, object_store_registry, } = self; - - // If `memory_pool` is not set, it represents spilling is disabled. - // Because `UnboundedMemoryPool` will be used in this case, and - // it will never limit memory usage. - // And we disable the `disk_manager`(use `DiskManagerConfig::Disabled`) - // to tell we have disabled spilling. - let (memory_pool, disk_manager_config) = if let Some(pool) = memory_pool { - (pool, disk_manager) - } else { - ( - Arc::new(UnboundedMemoryPool::default()) as _, - DiskManagerConfig::Disabled, - ) - }; + let memory_pool = + memory_pool.unwrap_or_else(|| Arc::new(UnboundedMemoryPool::default())); Ok(RuntimeEnv { memory_pool, - disk_manager: DiskManager::try_new(disk_manager_config)?, + disk_manager: DiskManager::try_new(disk_manager)?, cache_manager: CacheManager::try_new(&cache_manager)?, object_store_registry, }) From 266b48ec30b0523212f11534891db1c7801790f6 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 22 Apr 2025 14:28:47 +0800 Subject: [PATCH 22/55] modify the spilling judgement. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 942ff27a7890..b1e5fb5ca8dc 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -41,7 +41,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::proxy::VecAllocExt; -use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryLimit, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; @@ -683,7 +683,7 @@ fn maybe_enable_blocked_groups( .enable_aggregation_blocked_groups || !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() - || context.runtime_env().disk_manager.tmp_files_enabled() + || matches!(context.memory_pool().memory_limit(), MemoryLimit::Infinite) { return Ok(false); } From 55de98caf2faac117fa7db3ab8e2235793b443d8 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 23 Apr 2025 02:23:16 +0800 Subject: [PATCH 23/55] add unit tests for primitive group values. --- .../group_values/single_group_by/primitive.rs | 169 ++++++++++++++++++ 1 file changed, 169 insertions(+) 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 252940fa753b..bf6fc1ab9d8f 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 @@ -408,3 +408,172 @@ where Ok(()) } } + +#[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, Int64Array, NullBufferBuilder, UInt32Array}; + use arrow::datatypes::{DataType, UInt32Type}; + use datafusion_expr::EmitTo; + use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + 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(&vec![data1.clone() 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(&vec![data2.clone() 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(&vec![data1.clone() 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(&vec![data2.clone() 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); + } +} From 914583356a3f35b6690ac26a4b45d22dc16a27dc Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 23 Apr 2025 22:53:11 +0800 Subject: [PATCH 24/55] improve comments for `GroupIndexOperations`. --- .../src/aggregate/groups_accumulator.rs | 31 +++++++++++++++++-- 1 file changed, 28 insertions(+), 3 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 86ca22c368f6..2c6f6241c475 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -514,9 +514,34 @@ pub(crate) fn slice_and_maybe_filter( // Useful tools for group index // =============================================== -/// Blocked style group index used in blocked mode group values and accumulators -/// - High 32 bits represent `block_id` -/// - Low 32 bits represent `block_offset` +/// 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; From 04f15b027a289e029fb96379c8725f146de6b570 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 24 Apr 2025 19:57:26 +0800 Subject: [PATCH 25/55] add `enable_aggregation_blocked_groups` to aggr fuzzy test. --- .../aggregation_fuzzer/context_generator.rs | 11 +++++++++++ 1 file changed, 11 insertions(+) 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 3c9fe2917251..5d37b5b29f47 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/context_generator.rs @@ -103,6 +103,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), }; @@ -146,11 +147,14 @@ impl SessionContextGenerator { (provider, false) }; + let enable_aggregation_blocked_groups = rng.gen_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), }; @@ -174,6 +178,7 @@ struct GeneratedSessionContextBuilder { target_partitions: usize, sort_hint: bool, skip_partial_params: SkipPartialParams, + enable_aggregation_blocked_groups: bool, table_name: String, table_provider: Arc, } @@ -198,6 +203,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)?; @@ -207,6 +216,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 }) @@ -221,6 +231,7 @@ pub struct SessionContextParams { target_partitions: usize, sort_hint: bool, skip_partial_params: SkipPartialParams, + enable_aggregation_blocked_groups: bool, } /// Partial skipping parameters From be64a741431212f788e1564b81bab002aabb02c6 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 25 Apr 2025 13:40:18 +0800 Subject: [PATCH 26/55] refactor and make `QueryBuilder` more configurable. --- .../fuzz_cases/aggregation_fuzzer/fuzzer.rs | 80 ++++++++++++++++++- 1 file changed, 78 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs index 53e9288ab4af..ca641344daee 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs @@ -380,13 +380,89 @@ fn format_batches_with_limit(batches: &[RecordBatch]) -> impl std::fmt::Display ///``` #[derive(Debug, Default, Clone)] pub struct QueryBuilder { + // =================================== + // Table settings + // =================================== /// The name of the table to query table_name: String, + + // =================================== + // Grouping settings + // =================================== + /// Columns to be used in randomly generate `groupings` + /// + /// # Example + /// + /// Columns: + /// + /// ```text + /// [a,b,c,d] + /// ``` + /// + /// And randomly generated `groupings` (at least 1 column) + /// can be: + /// + /// ```text + /// [a] + /// [a,b] + /// [a,b,d] + /// ... + /// ``` + /// + /// So the finally generated sqls will be: + /// + /// ```text + /// SELECT aggr FROM t GROUP BY a; + /// SELECT aggr FROM t GROUP BY a,b; + /// SELECT aggr FROM t GROUP BY a,b,d; + /// ... + /// ``` + random_grouping_columns: Vec, + + /// Max columns num in randomly generated `groupings` + max_random_grouping_columns: usize, + + /// The sort keys of dataset + /// + /// Due to optimizations will be triggered when all or some + /// grouping columns are the sort keys of dataset. + /// So it is necessary to randomly generate some `groupings` basing on + /// dataset sort keys for test coverage. + /// + /// # Example + /// + /// Dataset including columns [a,b,c], and sorted by [a,b] + /// + /// And we may generate sqls to try covering the sort-optimization cases like: + /// + /// ```text + /// SELECT aggr FROM t GROUP BY b; // no permutation case + /// SELECT aggr FROM t GROUP BY a,c; // partial permutation case + /// SELECT aggr FROM t GROUP BY a,b,c; // full permutation case + /// ... + /// ``` + /// + /// More details can see [`GroupOrdering`]. + /// + /// [`GroupOrdering`]: datafusion_physical_plan::aggregates::order::GroupOrdering + /// + dataset_sort_keys: Vec>, + + /// If we will also test the no grouping case like: + /// + /// ```text + /// SELECT aggr FROM t; + /// ``` + /// + no_grouping: bool, + + // ==================================== + // Aggregation function settings + // ==================================== /// Aggregate functions to be used in the query /// (function_name, is_distinct) aggregate_functions: Vec<(String, bool)>, - /// Columns to be used in group by - group_by_columns: Vec, + /// Possible columns for arguments in the aggregate functions /// /// Assumes each From 7da0259686a454afa301f4f82a7a7c5f4e3ab49a Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 25 Apr 2025 14:47:37 +0800 Subject: [PATCH 27/55] fix tests. --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 7 ++ .../fuzz_cases/aggregation_fuzzer/fuzzer.rs | 102 +++++++++++++----- 2 files changed, 80 insertions(+), 29 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index ff3b66986ced..958a9d82fe3b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -85,6 +85,7 @@ async fn test_min() { .with_aggregate_function("min") // min works on all column types .with_aggregate_arguments(data_gen_config.all_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) @@ -111,6 +112,7 @@ async fn test_first_val() { .with_table_name("fuzz_table") .with_aggregate_function("first_value") .with_aggregate_arguments(data_gen_config.all_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) @@ -137,6 +139,7 @@ async fn test_last_val() { .with_table_name("fuzz_table") .with_aggregate_function("last_value") .with_aggregate_arguments(data_gen_config.all_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) @@ -156,6 +159,7 @@ async fn test_max() { .with_aggregate_function("max") // max works on all column types .with_aggregate_arguments(data_gen_config.all_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) @@ -176,6 +180,7 @@ async fn test_sum() { .with_distinct_aggregate_function("sum") // sum only works on numeric columns .with_aggregate_arguments(data_gen_config.numeric_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) @@ -196,6 +201,7 @@ async fn test_count() { .with_distinct_aggregate_function("count") // count work for all arguments .with_aggregate_arguments(data_gen_config.all_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) @@ -216,6 +222,7 @@ async fn test_median() { .with_distinct_aggregate_function("median") // median only works on numeric columns .with_aggregate_arguments(data_gen_config.numeric_columns()) + .with_dataset_sort_keys(data_gen_config.sort_keys_set.clone()) .set_group_by_columns(data_gen_config.all_columns()); AggregationFuzzerBuilder::from(data_gen_config) diff --git a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs index ca641344daee..7af9a5b4c022 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/fuzzer.rs @@ -69,30 +69,16 @@ impl AggregationFuzzerBuilder { /// - 3 random queries /// - 3 random queries for each group by selected from the sort keys /// - 1 random query with no grouping - pub fn add_query_builder(mut self, mut query_builder: QueryBuilder) -> Self { - const NUM_QUERIES: usize = 3; - for _ in 0..NUM_QUERIES { - let sql = query_builder.generate_query(); - self.candidate_sqls.push(Arc::from(sql)); - } - // also add several queries limited to grouping on the group by columns only, if any - // So if the data is sorted on `a,b` only group by `a,b` or`a` or `b` - if let Some(data_gen_config) = &self.data_gen_config { - for sort_keys in &data_gen_config.sort_keys_set { - let group_by_columns = sort_keys.iter().map(|s| s.as_str()); - query_builder = query_builder.set_group_by_columns(group_by_columns); - for _ in 0..NUM_QUERIES { - let sql = query_builder.generate_query(); - self.candidate_sqls.push(Arc::from(sql)); - } - } - } - // also add a query with no grouping - query_builder = query_builder.set_group_by_columns(vec![]); - let sql = query_builder.generate_query(); - self.candidate_sqls.push(Arc::from(sql)); + pub fn add_query_builder(mut self, query_builder: QueryBuilder) -> Self { + self = self.table_name(query_builder.table_name()); + + let sqls = query_builder + .generate_queries() + .into_iter() + .map(|sql| Arc::from(sql.as_str())); + self.candidate_sqls.extend(sqls); - self.table_name(query_builder.table_name()) + self } pub fn table_name(mut self, table_name: &str) -> Self { @@ -417,10 +403,10 @@ pub struct QueryBuilder { /// SELECT aggr FROM t GROUP BY a,b,d; /// ... /// ``` - random_grouping_columns: Vec, + group_by_columns: Vec, /// Max columns num in randomly generated `groupings` - max_random_grouping_columns: usize, + max_group_by_columns: usize, /// The sort keys of dataset /// @@ -470,7 +456,11 @@ pub struct QueryBuilder { } impl QueryBuilder { pub fn new() -> Self { - Default::default() + Self { + no_grouping: true, + max_group_by_columns: 3, + ..Default::default() + } } /// return the table name if any @@ -526,7 +516,62 @@ impl QueryBuilder { self } - pub fn generate_query(&self) -> String { + /// 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 + pub fn with_max_group_by_columns(mut self, group_by_columns: usize) -> Self { + self.max_group_by_columns = group_by_columns; + self + } + + /// Add sort keys of dataset if any, then the builder will generate queries basing on it + /// to cover the sort-optimization cases + pub fn with_dataset_sort_keys(mut self, dataset_sort_keys: Vec>) -> Self { + self.dataset_sort_keys = dataset_sort_keys; + self + } + + /// Add if also test the no grouping aggregation case(default: true) + pub fn with_no_grouping(mut self, no_grouping: bool) -> Self { + self.no_grouping = no_grouping; + self + } + + pub fn generate_queries(mut self) -> Vec { + const NUM_QUERIES: usize = 3; + let mut sqls = Vec::new(); + + // Add several queries group on randomly picked columns + for _ in 0..NUM_QUERIES { + let sql = self.generate_query(); + sqls.push(sql); + } + + // Also add several queries limited to grouping on the group by + // dataset sorted columns only, if any. + // So if the data is sorted on `a,b` only group by `a,b` or`a` or `b`. + if !self.dataset_sort_keys.is_empty() { + let dataset_sort_keys = self.dataset_sort_keys.clone(); + for sort_keys in dataset_sort_keys { + let group_by_columns = sort_keys.iter().map(|s| s.as_str()); + self = self.set_group_by_columns(group_by_columns); + for _ in 0..NUM_QUERIES { + let sql = self.generate_query(); + sqls.push(sql); + } + } + } + + // Also add a query with no grouping + if self.no_grouping { + self = self.set_group_by_columns(vec![]); + let sql = self.generate_query(); + sqls.push(sql); + } + + sqls + } + + fn generate_query(&self) -> String { let group_by = self.random_group_by(); let mut query = String::from("SELECT "); query.push_str(&group_by.join(", ")); @@ -644,8 +689,7 @@ impl QueryBuilder { /// larger numbers of columns, each group has many fewer values. fn random_group_by(&self) -> Vec { let mut rng = thread_rng(); - const MAX_GROUPS: usize = 3; - let max_groups = self.group_by_columns.len().max(MAX_GROUPS); + let max_groups = self.group_by_columns.len().max(self.max_group_by_columns); let num_group_by = rng.gen_range(1..max_groups); let mut already_used = HashSet::new(); From ffb11cd1ec1ca049995b0ed567b450a5e7b6c6d0 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 27 Apr 2025 10:30:21 +0800 Subject: [PATCH 28/55] fix clippy. --- .../aggregates/group_values/single_group_by/primitive.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 bf6fc1ab9d8f..5ebe04586580 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 @@ -450,7 +450,7 @@ mod tests { // Insert case 1.1, 1.3, 1.4 + Emit case 2.1 group_values - .intern(&vec![data1.clone() as _], &mut group_indices) + .intern(&vec![Arc::clone(&data1) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); @@ -477,7 +477,7 @@ mod tests { // Insert case 1.1~1.3 + Emit case 2.2~2.3 group_values - .intern(&vec![data2.clone() as _], &mut group_indices) + .intern(&vec![Arc::clone(&data2) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); @@ -528,7 +528,7 @@ mod tests { // Insert case 1.1, 1.3, 1.4 + Emit case 2.1 group_values - .intern(&vec![data1.clone() as _], &mut group_indices) + .intern(&vec![Arc::clone(&data1) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); @@ -553,7 +553,7 @@ mod tests { // Insert case 1.1~1.2 + Emit case 2.2 group_values - .intern(&vec![data2.clone() as _], &mut group_indices) + .intern(&vec![Arc::clone(&data2) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); From 7f543d8a6dce1c3d03ada6b09d7768d220f3179c Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 27 Apr 2025 14:19:44 +0800 Subject: [PATCH 29/55] add fuzzy tests for blocked groups, and fix enable logic. --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 40 +++++++++++++++++++ .../aggregation_fuzzer/query_builder.rs | 1 - .../physical-plan/src/aggregates/row_hash.rs | 2 +- 3 files changed, 41 insertions(+), 2 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 8c70b02a54fe..d0b948ce2e5f 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -233,6 +233,46 @@ async fn test_median() { .await; } +// Testing `blocked groups optimization` +// Details of this optimization can see: +// https://github.com/apache/datafusion/issues/7065 +#[tokio::test(flavor = "multi_thread")] +async fn test_blocked_groups_optimization() { + let data_gen_config = baseline_config(); + + // Blocked groups supporting lists: + // + // `GroupAccumulator`: + // - PrimitiveGroupsAccumulator + // + // `GroupValues`: + // - GroupValuesPrimitive + // + + // 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) + .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/query_builder.rs b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs index df4730214f1a..e1efffbca814 100644 --- a/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs +++ b/datafusion/core/tests/fuzz_cases/aggregation_fuzzer/query_builder.rs @@ -247,7 +247,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/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index b1e5fb5ca8dc..164d9116ef14 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -683,7 +683,7 @@ fn maybe_enable_blocked_groups( .enable_aggregation_blocked_groups || !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() - || matches!(context.memory_pool().memory_limit(), MemoryLimit::Infinite) + || !matches!(context.memory_pool().memory_limit(), MemoryLimit::Infinite) { return Ok(false); } From 868210faa2fbd2f302ffae456c8258c9743a7d68 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 27 Apr 2025 20:51:28 +0800 Subject: [PATCH 30/55] update config.md and fix clippy. --- Cargo.lock | 4 +- .../src/aggregate/groups_accumulator.rs | 4 +- .../groups_accumulator/accumulate.rs | 19 +- .../aggregate/groups_accumulator/prim_op.rs | 2 +- .../group_values/single_group_by/primitive.rs | 10 +- docs/source/user-guide/configs.md | 195 +++++++++--------- 6 files changed, 119 insertions(+), 115 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 48b8689de0ae..63d0f7619cd4 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2806,9 +2806,9 @@ dependencies = [ [[package]] name = "env_logger" -version = "0.11.7" +version = "0.11.8" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c3716d7a920fb4fac5d84e9d4bce8ceb321e9414b4409da61b07b75c1e3d0697" +checksum = "13c863f0904021b108aa8b2f55046443e6b1ebde8fd4a15c399893aae4fa069f" dependencies = [ "anstream", "anstyle", diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 2c6f6241c475..af210b587378 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -601,7 +601,7 @@ pub(crate) fn ensure_room_enough_for_blocks( // For resize, we need to: // 1. Ensure the blks are enough first // 2. and then ensure slots in blks are enough - let (mut cur_blk_idx, exist_slots) = if blocks.len() > 0 { + let (mut cur_blk_idx, exist_slots) = if !blocks.is_empty() { let cur_blk_idx = blocks.len() - 1; let exist_slots = (blocks.len() - 1) * block_size + blocks.back().unwrap().len(); @@ -617,7 +617,7 @@ pub(crate) fn ensure_room_enough_for_blocks( // 1. Ensure blks are enough let exist_blks = blocks.len(); - let new_blks = ((total_num_groups + block_size - 1) / block_size) - exist_blks; + let new_blks = total_num_groups.div_ceil(block_size) - exist_blks; if new_blks > 0 { for _ in 0..new_blks { let block = new_block(block_size); 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 6d78ef4e2555..02e9807c4e2d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -527,17 +527,22 @@ impl NullStateAdapter { } } +/// [`NullState`] implementation for `flat group index` pub type FlatNullState = NullState; impl FlatNullState { pub fn new() -> Self { - Self { - seen_values: FlatSeenValues::default(), - _phantom: PhantomData {}, - } + Self::default() + } +} + +impl Default for FlatNullState { + fn default() -> Self { + Self::new() } } +/// [`NullState`] implementation for `blocked group index` pub type BlockedNullState = NullState; impl BlockedNullState { @@ -1210,7 +1215,6 @@ mod test { let group_indices_chunks = group_indices .chunks(chunk_size) - .into_iter() .map(|chunk| chunk.to_vec()) .collect::>(); @@ -1295,7 +1299,7 @@ mod test { let flatten_index = ((block_id as u64 * block_size as u64) + block_offset) as usize; - accumulated_values.push((flatten_index as usize, value)); + accumulated_values.push((flatten_index, value)); }, ); } @@ -1413,7 +1417,6 @@ mod test { let group_indices_chunks = group_indices .chunks(chunk_size) - .into_iter() .map(|chunk| chunk.to_vec()) .collect::>(); @@ -1498,7 +1501,7 @@ mod test { let flatten_index = ((block_id as u64 * block_size as u64) + block_offset) as usize; - accumulated_values.push((flatten_index as usize, value)); + accumulated_values.push((flatten_index, value)); }, ); } 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 003831277ef3..dc92408e6dd8 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 @@ -254,6 +254,6 @@ impl Block for Vec { } fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { - self.extend(iter::repeat(default_value.clone()).take(fill_len)); + self.extend(iter::repeat_n(default_value, fill_len)); } } 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 5ebe04586580..307a149afa25 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 @@ -416,7 +416,7 @@ mod tests { use crate::aggregates::group_values::single_group_by::primitive::GroupValuesPrimitive; use crate::aggregates::group_values::GroupValues; - use arrow::array::{AsArray, Int64Array, NullBufferBuilder, UInt32Array}; + use arrow::array::{AsArray, UInt32Array}; use arrow::datatypes::{DataType, UInt32Type}; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ @@ -450,7 +450,7 @@ mod tests { // Insert case 1.1, 1.3, 1.4 + Emit case 2.1 group_values - .intern(&vec![Arc::clone(&data1) as _], &mut group_indices) + .intern(&[Arc::clone(&data1) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); @@ -477,7 +477,7 @@ mod tests { // Insert case 1.1~1.3 + Emit case 2.2~2.3 group_values - .intern(&vec![Arc::clone(&data2) as _], &mut group_indices) + .intern(&[Arc::clone(&data2) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); @@ -528,7 +528,7 @@ mod tests { // Insert case 1.1, 1.3, 1.4 + Emit case 2.1 group_values - .intern(&vec![Arc::clone(&data1) as _], &mut group_indices) + .intern(&[Arc::clone(&data1) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); @@ -553,7 +553,7 @@ mod tests { // Insert case 1.1~1.2 + Emit case 2.2 group_values - .intern(&vec![Arc::clone(&data2) as _], &mut group_indices) + .intern(&[Arc::clone(&data2) as _], &mut group_indices) .unwrap(); let mut expected = BTreeMap::new(); diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 7a46d59d893e..bb2456b018f4 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,100 +35,101 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| 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 | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | -| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 47.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_read | true | (writing) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| 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.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. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | -| datafusion.sql_parser.map_varchar_to_utf8view | false | If true, `VARCHAR` is mapped to `Utf8View` during SQL planning. If false, `VARCHAR` is mapped to `Utf8` during SQL planning. Default is false. | -| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | -| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | +| key | default | description | +| ----------------------------------------------------------------------- | ------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| 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 | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | +| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 47.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_read | true | (writing) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| 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.enable_aggregation_blocked_groups | true | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the blocked approach will be used. And the blocked approach allocates capacity for the block 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 setting this flag to `false`, will fall-back to use the single approach, values 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. | +| 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. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| datafusion.sql_parser.map_varchar_to_utf8view | false | If true, `VARCHAR` is mapped to `Utf8View` during SQL planning. If false, `VARCHAR` is mapped to `Utf8` during SQL planning. Default is false. | +| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | +| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | From bdcd1b8ac37bcb29c829df71625716ba161dcff4 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 27 Apr 2025 22:48:39 +0800 Subject: [PATCH 31/55] improve comment about blocked groups for `GroupedHashAggregateStream`. --- .../physical-plan/src/aggregates/row_hash.rs | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 164d9116ef14..0706050b73d2 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -344,6 +344,31 @@ impl SkipAggregationProbe { /// │ 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. +/// For a nice introduction to the blocked approach, maybe you can see [#7065]. +/// +/// The conditions that trigger the blocked groups optimization can be found in +/// [`maybe_enable_blocked_groups`]. +/// +/// [`group_values`]: Self::group_values +/// [`accumulators`]: Self::accumulators +/// [#7065]: https://github.com/apache/datafusion/issues/7065 +/// pub(crate) struct GroupedHashAggregateStream { // ======================================================================== // PROPERTIES: From 3c7317df1ce752f6091481530061fb578d7fe1da Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 27 Apr 2025 23:42:08 +0800 Subject: [PATCH 32/55] fix stack overflow. --- .../src/aggregate/groups_accumulator/accumulate.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 02e9807c4e2d..5ca5e1525591 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -538,7 +538,10 @@ impl FlatNullState { impl Default for FlatNullState { fn default() -> Self { - Self::new() + Self { + seen_values: FlatSeenValues::default(), + _phantom: PhantomData {}, + } } } From ff9c3adc4523e7df83ccccee635679926f083dda Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 27 Apr 2025 23:51:18 +0800 Subject: [PATCH 33/55] add extended query to see the improvement. --- benchmarks/queries/clickbench/extended.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/benchmarks/queries/clickbench/extended.sql b/benchmarks/queries/clickbench/extended.sql index e967583fd644..1885b2b57c92 100644 --- a/benchmarks/queries/clickbench/extended.sql +++ b/benchmarks/queries/clickbench/extended.sql @@ -5,3 +5,4 @@ SELECT "SocialSourceNetworkID", "RegionID", COUNT(*), AVG("Age"), AVG("ParamPric SELECT "ClientIP", "WatchID", COUNT(*) c, MIN("ResponseStartTiming") tmin, MEDIAN("ResponseStartTiming") tmed, MAX("ResponseStartTiming") tmax FROM hits WHERE "JavaEnable" = 0 GROUP BY "ClientIP", "WatchID" HAVING c > 1 ORDER BY tmed DESC LIMIT 10; SELECT "ClientIP", "WatchID", COUNT(*) c, MIN("ResponseStartTiming") tmin, APPROX_PERCENTILE_CONT("ResponseStartTiming", 0.95) tp95, MAX("ResponseStartTiming") tmax FROM 'hits' WHERE "JavaEnable" = 0 GROUP BY "ClientIP", "WatchID" HAVING c > 1 ORDER BY tp95 DESC LIMIT 10; SELECT COUNT(*) AS ShareCount FROM hits WHERE "IsMobile" = 1 AND "MobilePhoneModel" LIKE 'iPhone%' AND "SocialAction" = 'share' AND "SocialSourceNetworkID" IN (5, 12) AND "ClientTimeZone" BETWEEN -5 AND 5 AND regexp_match("Referer", '\/campaign\/(spring|summer)_promo') IS NOT NULL AND CASE WHEN split_part(split_part("URL", 'resolution=', 2), '&', 1) ~ '^\d+$' THEN split_part(split_part("URL", 'resolution=', 2), '&', 1)::INT ELSE 0 END > 1920 AND levenshtein(CAST("UTMSource" AS STRING), CAST("UTMCampaign" AS STRING)) < 3; +SELECT "WatchID", MIN("ResolutionWidth"), MAX("ResolutionWidth"), SUM("IsRefresh") FROM hits GROUP BY "WatchID" ORDER BY "WatchID" DESC LIMIT 10; From bb63628c3ceeb670be741e1927af51a5bb3b6e35 Mon Sep 17 00:00:00 2001 From: kamille <3144148605@qq.com> Date: Sat, 3 May 2025 23:27:01 +0800 Subject: [PATCH 34/55] Update datafusion/common/src/config.rs Co-authored-by: Andrew Lamb --- datafusion/common/src/config.rs | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 977f6ae12f74..eafac7f4ee99 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -405,15 +405,14 @@ config_namespace! { /// tables with a highly-selective join filter, but is also slightly slower. pub enforce_batch_size_in_joins: bool, default = false - /// Should DataFusion use the the blocked approach to manage the groups - /// values and their related states in accumulators. - /// By default, the blocked approach will be used. And the blocked approach - /// allocates capacity for the block based on a predefined block size firstly. + /// 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 setting this flag to `false`, will fall-back to use the single approach, - /// values are managed within a single large block(can think of it as a Vec). + /// 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 From a7c4c7b774e0f62f3f8492244fa23c2fc5528084 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 00:16:36 +0800 Subject: [PATCH 35/55] update config. --- docs/source/user-guide/configs.md | 216 +++++++++++++++--------------- 1 file changed, 108 insertions(+), 108 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index a011023e4f03..0a844b57ba7a 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,111 +35,111 @@ Values are parsed according to the [same rules used in casts from Utf8](https:// If the value in the environment variable cannot be cast to the type of the configuration option, the default value will be used instead and a warning emitted. Environment variables are read during `SessionConfig` initialisation so they must be set beforehand and will not affect running sessions. -| key | default | description | -| ----------------------------------------------------------------------- | ------------------------- | --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- | -| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | -| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | -| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | -| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | -| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | -| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | -| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | -| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | -| 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 | false | Should DataFusion collect statistics after listing files | -| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | -| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | -| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | -| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | -| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | -| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | -| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | -| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | -| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | -| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | -| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | -| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | -| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | -| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | -| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | -| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | -| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | -| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | -| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | -| datafusion.execution.parquet.created_by | datafusion version 47.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | -| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | -| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | -| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | -| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | -| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | -| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | -| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | -| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | -| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | -| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | -| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | -| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | -| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | -| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | -| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | -| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | -| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | -| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | -| 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.enable_aggregation_blocked_groups | true | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the blocked approach will be used. And the blocked approach allocates capacity for the block 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 setting this flag to `false`, will fall-back to use the single approach, values 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. | -| 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.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. | -| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | -| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | -| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | -| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | -| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | -| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | -| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | -| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | -| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | -| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | -| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | -| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | -| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | -| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | -| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | -| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | -| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | -| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | -| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | -| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | -| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | -| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | -| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | -| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | -| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | -| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | -| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | -| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | -| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | -| datafusion.sql_parser.map_varchar_to_utf8view | false | If true, `VARCHAR` is mapped to `Utf8View` during SQL planning. If false, `VARCHAR` is mapped to `Utf8` during SQL planning. Default is false. | -| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | -| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | -| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | -| datafusion.format.null | | Format string for nulls | -| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | -| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | -| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | -| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | -| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | -| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | -| datafusion.format.types_info | false | Show types in visual representation batches | +| key | default | description | +| ----------------------------------------------------------------------- | ------------------------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ | +| datafusion.catalog.create_default_catalog_and_schema | true | Whether the default catalog and schema should be created automatically. | +| datafusion.catalog.default_catalog | datafusion | The default catalog name - this impacts what SQL queries use if not specified | +| datafusion.catalog.default_schema | public | The default schema name - this impacts what SQL queries use if not specified | +| datafusion.catalog.information_schema | false | Should DataFusion provide access to `information_schema` virtual tables for displaying schema information | +| datafusion.catalog.location | NULL | Location scanned to load tables for `default` schema | +| datafusion.catalog.format | NULL | Type of `TableProvider` to use when loading `default` schema | +| datafusion.catalog.has_header | true | Default value for `format.has_header` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. | +| datafusion.catalog.newlines_in_values | false | Specifies whether newlines in (quoted) CSV values are supported. This is the default value for `format.newlines_in_values` for `CREATE EXTERNAL TABLE` if not specified explicitly in the statement. Parsing newlines in quoted values may be affected by execution behaviour such as parallel file scanning. Setting this to `true` ensures that newlines in values are parsed successfully, which may reduce performance. | +| 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 | false | Should DataFusion collect statistics after listing files | +| datafusion.execution.target_partitions | 0 | Number of partitions for query execution. Increasing partitions can increase concurrency. Defaults to the number of CPU cores on the system | +| datafusion.execution.time_zone | +00:00 | The default time zone Some functions, e.g. `EXTRACT(HOUR from SOME_TIME)`, shift the underlying datetime according to this time zone, and then extract the hour | +| datafusion.execution.parquet.enable_page_index | true | (reading) If true, reads the Parquet data page level metadata (the Page Index), if present, to reduce the I/O and number of rows decoded. | +| datafusion.execution.parquet.pruning | true | (reading) If true, the parquet reader attempts to skip entire row groups based on the predicate in the query and the metadata (min/max values) stored in the parquet file | +| datafusion.execution.parquet.skip_metadata | true | (reading) If true, the parquet reader skip the optional embedded metadata that may be in the file Schema. This setting can help avoid schema conflicts when querying multiple parquet files with schemas containing compatible types but different metadata | +| datafusion.execution.parquet.metadata_size_hint | NULL | (reading) If specified, the parquet reader will try and fetch the last `size_hint` bytes of the parquet file optimistically. If not specified, two reads are required: One read to fetch the 8-byte parquet footer and another to fetch the metadata length encoded in the footer | +| datafusion.execution.parquet.pushdown_filters | false | (reading) If true, filter expressions are be applied during the parquet decoding operation to reduce the number of rows decoded. This optimization is sometimes called "late materialization". | +| datafusion.execution.parquet.reorder_filters | false | (reading) If true, filter expressions evaluated during the parquet decoding operation will be reordered heuristically to minimize the cost of evaluation. If false, the filters are applied in the same order as written in the query | +| datafusion.execution.parquet.schema_force_view_types | true | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.parquet.binary_as_string | false | (reading) If true, parquet reader will read columns of `Binary/LargeBinary` with `Utf8`, and `BinaryView` with `Utf8View`. Parquet files generated by some legacy writers do not correctly set the UTF8 flag for strings, causing string columns to be loaded as BLOB instead. | +| datafusion.execution.parquet.coerce_int96 | NULL | (reading) If true, parquet reader will read columns of physical type int96 as originating from a different resolution than nanosecond. This is useful for reading data from systems like Spark which stores microsecond resolution timestamps in an int96 allowing it to write values with a larger date range than 64-bit timestamps with nanosecond resolution. | +| datafusion.execution.parquet.bloom_filter_on_read | true | (reading) Use any available bloom filters when reading parquet files | +| datafusion.execution.parquet.data_pagesize_limit | 1048576 | (writing) Sets best effort maximum size of data page in bytes | +| datafusion.execution.parquet.write_batch_size | 1024 | (writing) Sets write_batch_size in bytes | +| datafusion.execution.parquet.writer_version | 1.0 | (writing) Sets parquet writer version valid values are "1.0" and "2.0" | +| datafusion.execution.parquet.skip_arrow_metadata | false | (writing) Skip encoding the embedded arrow metadata in the KV_meta This is analogous to the `ArrowWriterOptions::with_skip_arrow_metadata`. Refer to | +| datafusion.execution.parquet.compression | zstd(3) | (writing) Sets default parquet compression codec. Valid values are: uncompressed, snappy, gzip(level), lzo, brotli(level), lz4, zstd(level), and lz4_raw. These values are not case sensitive. If NULL, uses default parquet writer setting Note that this default setting is not the same as the default parquet writer setting. | +| datafusion.execution.parquet.dictionary_enabled | true | (writing) Sets if dictionary encoding is enabled. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.dictionary_page_size_limit | 1048576 | (writing) Sets best effort maximum dictionary page size, in bytes | +| datafusion.execution.parquet.statistics_enabled | page | (writing) Sets if statistics are enabled for any column Valid values are: "none", "chunk", and "page" These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.max_statistics_size | 4096 | (writing) Sets max statistics size for any column. If NULL, uses default parquet writer setting max_statistics_size is deprecated, currently it is not being used | +| datafusion.execution.parquet.max_row_group_size | 1048576 | (writing) Target maximum number of rows in each row group (defaults to 1M rows). Writing larger row groups requires more memory to write, but can get better compression and be faster to read. | +| datafusion.execution.parquet.created_by | datafusion version 47.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| datafusion.execution.parquet.statistics_truncate_length | NULL | (writing) Sets statictics truncate length. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.data_page_row_count_limit | 20000 | (writing) Sets best effort maximum number of rows in data page | +| datafusion.execution.parquet.encoding | NULL | (writing) Sets default encoding for any column. Valid values are: plain, plain_dictionary, rle, bit_packed, delta_binary_packed, delta_length_byte_array, delta_byte_array, rle_dictionary, and byte_stream_split. These values are not case sensitive. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_on_write | false | (writing) Write bloom filters for all columns when creating parquet files | +| datafusion.execution.parquet.bloom_filter_fpp | NULL | (writing) Sets bloom filter false positive probability. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.bloom_filter_ndv | NULL | (writing) Sets bloom filter number of distinct values. If NULL, uses default parquet writer setting | +| datafusion.execution.parquet.allow_single_file_parallelism | true | (writing) Controls whether DataFusion will attempt to speed up writing parquet files by serializing them in parallel. Each column in each row group in each output file are serialized in parallel leveraging a maximum possible core count of n_files*n_row_groups*n_columns. | +| datafusion.execution.parquet.maximum_parallel_row_group_writers | 1 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.parquet.maximum_buffered_record_batches_per_stream | 2 | (writing) By default parallel parquet writer is tuned for minimum memory usage in a streaming execution plan. You may see a performance benefit when writing large parquet files by increasing maximum_parallel_row_group_writers and maximum_buffered_record_batches_per_stream if your system has idle cores and can tolerate additional memory usage. Boosting these values is likely worthwhile when writing out already in-memory data, such as from a cached data frame. | +| datafusion.execution.planning_concurrency | 0 | Fan-out during initial physical planning. This is mostly use to plan `UNION` children in parallel. Defaults to the number of CPU cores on the system | +| datafusion.execution.skip_physical_aggregate_schema_check | false | When set to true, skips verifying that the schema produced by planning the input of `LogicalPlan::Aggregate` exactly matches the schema of the input plan. When set to false, if the schema does not match exactly (including nullability and metadata), a planning error will be raised. This is used to workaround bugs in the planner that are now caught by the new schema verification step. | +| datafusion.execution.sort_spill_reservation_bytes | 10485760 | Specifies the reserved memory for each spillable sort operation to facilitate an in-memory merge. When a sort operation spills to disk, the in-memory data must be sorted and merged before being written to a file. This setting reserves a specific amount of memory for that in-memory sort/merge process. Note: This setting is irrelevant if the sort operation cannot spill (i.e., if there's no `DiskManager` configured). | +| datafusion.execution.sort_in_place_threshold_bytes | 1048576 | When sorting, below what size should data be concatenated and sorted in a single RecordBatch rather than sorted in batches and merged. | +| datafusion.execution.meta_fetch_concurrency | 32 | Number of files to read in parallel when inferring schema and statistics | +| datafusion.execution.minimum_parallel_output_files | 4 | Guarantees a minimum level of output files running in parallel. RecordBatches will be distributed in round robin fashion to each parallel writer. Each writer is closed and a new file opened once soft_max_rows_per_output_file is reached. | +| datafusion.execution.soft_max_rows_per_output_file | 50000000 | Target number of rows in output files when writing multiple. This is a soft max, so it can be exceeded slightly. There also will be one file smaller than the limit if the total number of rows written is not roughly divisible by the soft max | +| datafusion.execution.max_buffered_batches_per_output_file | 2 | This is the maximum number of RecordBatches buffered for each output file being worked. Higher values can potentially give faster write performance at the cost of higher peak memory consumption | +| datafusion.execution.listing_table_ignore_subdirectory | true | Should sub directories be ignored when scanning directories for data files. Defaults to true (ignores subdirectories), consistent with Hive. Note that this setting does not affect reading partitioned tables (e.g. `/table/year=2021/month=01/data.parquet`). | +| datafusion.execution.enable_recursive_ctes | true | Should DataFusion support recursive CTEs | +| datafusion.execution.split_file_groups_by_statistics | false | Attempt to eliminate sorts by packing & sorting files with non-overlapping statistics into the same file groups. Currently experimental | +| datafusion.execution.keep_partition_by_columns | false | Should DataFusion keep the columns used for partition_by in the output RecordBatches | +| datafusion.execution.skip_partial_aggregation_probe_ratio_threshold | 0.8 | Aggregation ratio (number of distinct groups / number of input rows) threshold for skipping partial aggregation. If the value is greater then partial aggregation will skip aggregation for further input | +| datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | +| 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.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.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.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. | +| datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | +| datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | +| datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_file_min_size | 10485760 | Minimum total files size in bytes to perform file scan repartitioning. | +| datafusion.optimizer.repartition_joins | true | Should DataFusion repartition data using the join keys to execute joins in parallel using the provided `target_partitions` level | +| datafusion.optimizer.allow_symmetric_joins_without_pruning | true | Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. | +| datafusion.optimizer.repartition_file_scans | true | When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. | +| datafusion.optimizer.repartition_windows | true | Should DataFusion repartition data using the partitions keys to execute window functions in parallel using the provided `target_partitions` level | +| datafusion.optimizer.repartition_sorts | true | Should DataFusion execute sorts in a per-partition fashion and merge afterwards instead of coalescing first and sorting globally. With this flag is enabled, plans in the form below `text "SortExec: [a@0 ASC]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` would turn into the plan below which performs better in multithreaded environments `text "SortPreservingMergeExec: [a@0 ASC]", " SortExec: [a@0 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", ` | +| datafusion.optimizer.prefer_existing_sort | false | When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. | +| datafusion.optimizer.skip_failed_rules | false | When set to true, the logical plan optimizer will produce warning messages if any optimization rules produce errors and then proceed to the next rule. When set to false, any rules that produce errors will cause the query to fail | +| datafusion.optimizer.max_passes | 3 | Number of times that the optimizer will attempt to optimize the plan | +| datafusion.optimizer.top_down_join_key_reordering | true | When set to true, the physical plan optimizer will run a top down process to reorder the join keys | +| datafusion.optimizer.prefer_hash_join | true | When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory | +| datafusion.optimizer.hash_join_single_partition_threshold | 1048576 | The maximum estimated size in bytes for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.hash_join_single_partition_threshold_rows | 131072 | The maximum estimated size in rows for one input side of a HashJoin will be collected into a single partition | +| datafusion.optimizer.default_filter_selectivity | 20 | The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). | +| datafusion.optimizer.prefer_existing_union | false | When set to true, the optimizer will not attempt to convert Union to Interleave | +| datafusion.optimizer.expand_views_at_output | false | When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. | +| datafusion.explain.logical_plan_only | false | When set to true, the explain statement will only print logical plans | +| datafusion.explain.physical_plan_only | false | When set to true, the explain statement will only print physical plans | +| datafusion.explain.show_statistics | false | When set to true, the explain statement will print operator statistics for physical plans | +| datafusion.explain.show_sizes | true | When set to true, the explain statement will print the partition sizes | +| datafusion.explain.show_schema | false | When set to true, the explain statement will print schema information | +| datafusion.explain.format | indent | Display format of explain. Default is "indent". When set to "tree", it will print the plan in a tree-rendered format. | +| datafusion.sql_parser.parse_float_as_decimal | false | When set to true, SQL parser will parse float as decimal type | +| datafusion.sql_parser.enable_ident_normalization | true | When set to true, SQL parser will normalize ident (convert ident to lowercase when not quoted) | +| datafusion.sql_parser.enable_options_value_normalization | false | When set to true, SQL parser will normalize options value (convert value to lowercase). Note that this option is ignored and will be removed in the future. All case-insensitive values are normalized automatically. | +| datafusion.sql_parser.dialect | generic | Configure the SQL dialect used by DataFusion's parser; supported values include: Generic, MySQL, PostgreSQL, Hive, SQLite, Snowflake, Redshift, MsSQL, ClickHouse, BigQuery, Ansi, DuckDB and Databricks. | +| datafusion.sql_parser.support_varchar_with_length | true | If true, permit lengths for `VARCHAR` such as `VARCHAR(20)`, but ignore the length. If false, error if a `VARCHAR` with a length is specified. The Arrow type system does not have a notion of maximum string length and thus DataFusion can not enforce such limits. | +| datafusion.sql_parser.map_varchar_to_utf8view | false | If true, `VARCHAR` is mapped to `Utf8View` during SQL planning. If false, `VARCHAR` is mapped to `Utf8` during SQL planning. Default is false. | +| datafusion.sql_parser.collect_spans | false | When set to true, the source locations relative to the original SQL query (i.e. [`Span`](https://docs.rs/sqlparser/latest/sqlparser/tokenizer/struct.Span.html)) will be collected and recorded in the logical plan nodes. | +| datafusion.sql_parser.recursion_limit | 50 | Specifies the recursion depth limit when parsing complex SQL Queries | +| datafusion.format.safe | true | If set to `true` any formatting errors will be written to the output instead of being converted into a [`std::fmt::Error`] | +| datafusion.format.null | | Format string for nulls | +| datafusion.format.date_format | %Y-%m-%d | Date format for date arrays | +| datafusion.format.datetime_format | %Y-%m-%dT%H:%M:%S%.f | Format for DateTime arrays | +| datafusion.format.timestamp_format | %Y-%m-%dT%H:%M:%S%.f | Timestamp format for timestamp arrays | +| datafusion.format.timestamp_tz_format | NULL | Timestamp format for timestamp with timezone arrays. When `None`, ISO 8601 format is used. | +| datafusion.format.time_format | %H:%M:%S%.f | Time format for time arrays | +| datafusion.format.duration_format | pretty | Duration format. Can be either `"pretty"` or `"ISO8601"` | +| datafusion.format.types_info | false | Show types in visual representation batches | From e03356712cc76928b5f41c1156e276f7471412ef Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 00:20:25 +0800 Subject: [PATCH 36/55] fix fmt. --- datafusion/common/src/config.rs | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index eafac7f4ee99..c0ec5f5f5ce0 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -405,23 +405,23 @@ config_namespace! { /// tables with a highly-selective join filter, but is also slightly slower. pub enforce_batch_size_in_joins: bool, default = false - /// Should DataFusion use a blocked approach to manage grouping state. - /// By default, the blocked approach is used which + /// 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. + 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. + /// 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 - - /// 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. - pub objectstore_writer_buffer_size: usize, default = 10 * 1024 * 1024 } } From d173056bdf889f52e3f25d0b0ea2afb4006734cb Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 00:26:13 +0800 Subject: [PATCH 37/55] fix logic test. --- datafusion/sqllogictest/test_files/information_schema.slt | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index dabb81aea4b0..cb2665965fcc 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -328,7 +328,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 false Should DataFusion collect statistics after listing files -datafusion.execution.enable_aggregation_blocked_groups true Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the blocked approach will be used. And the blocked approach allocates capacity for the block 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 setting this flag to `false`, will fall-back to use the single approach, values 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. +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_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. datafusion.execution.keep_partition_by_columns false Should DataFusion keep the columns used for partition_by in the output RecordBatches @@ -676,16 +676,14 @@ DROP VIEW test.xyz # show_external_create_table() -statement ok +statement error DataFusion error: Object Store error: Object at location /Users/kamille/Desktop/github/datafusion/testing/data/csv/aggregate_test_100\.csv not found: No such file or directory \(os error 2\) CREATE EXTERNAL TABLE abc STORED AS CSV LOCATION '../../testing/data/csv/aggregate_test_100.csv' OPTIONS ('format.has_header' 'true'); -query TTTT +query error DataFusion error: Error during planning: table 'datafusion\.public\.abc' not found SHOW CREATE TABLE abc; ----- -datafusion public abc CREATE EXTERNAL TABLE abc STORED AS CSV LOCATION ../../testing/data/csv/aggregate_test_100.csv # string_agg has different arg_types but same return type. Test avoiding duplicate entries for the same function. query TTT From 426e2eeb5777f449cac4489f860c535263dab23c Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 00:43:57 +0800 Subject: [PATCH 38/55] improve comments. --- datafusion-examples/examples/advanced_udaf.rs | 8 +++++- .../expr-common/src/groups_accumulator.rs | 27 +++++++++++++++++++ .../groups_accumulator/accumulate.rs | 16 ++++++++--- datafusion/functions-aggregate/src/average.rs | 11 ++++++++ .../src/aggregates/group_values/mod.rs | 19 +++++++++++++ .../physical-plan/src/aggregates/row_hash.rs | 7 ++++- .../test_files/information_schema.slt | 6 +++-- docs/source/user-guide/configs.md | 2 +- 8 files changed, 88 insertions(+), 8 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 6d259a2dd371..727be7017491 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -248,7 +248,10 @@ 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, @@ -280,6 +283,9 @@ 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, diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 042013eaf7f7..508dd087df2d 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -41,6 +41,14 @@ pub enum EmitTo { impl EmitTo { /// Remove and return `needed values` from `values`. + /// + /// Inputs: + /// - `values`, the emitting source. + /// - `is_blocked_groups`, is the `values` organized in `single` + /// or `blocked` approach, more details can see + /// [`GroupsAccumulator::supports_blocked_groups`]. + /// + /// pub fn take_needed( &self, values: &mut VecDeque>, @@ -290,6 +298,25 @@ pub trait GroupsAccumulator: Send { 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: + /// https://github.com/apache/datafusion/issues/7065 + /// fn supports_blocked_groups(&self) -> bool { false } 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 5ca5e1525591..163411276b3f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -245,7 +245,10 @@ pub trait SeenValues: Default + Debug + Send { } /// [`SeenValues`] 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 @@ -261,7 +264,9 @@ pub trait SeenValues: Default + Debug + Send { /// /// 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 +/// #[derive(Debug)] pub struct FlatSeenValues { builder: BooleanBufferBuilder, @@ -318,6 +323,9 @@ impl SeenValues for FlatSeenValues { /// [`SeenValues`] 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 @@ -328,10 +336,12 @@ impl SeenValues for FlatSeenValues { /// row_n (block_id_n, block_offset_n) /// ``` /// -/// If ` row_x (block_id_x, block_offset_x)` is not filtered +/// 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 +/// #[derive(Debug, Default)] pub struct BlockedSeenValues { blocked_builders: VecDeque, diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index b1972e2d8696..5e7d77cdbba6 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -579,6 +579,10 @@ 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, @@ -663,6 +667,10 @@ 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, @@ -675,6 +683,9 @@ where // 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, diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index a4d91933b5a7..ffe110502ada 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -112,6 +112,25 @@ pub(crate) trait GroupValues: Send { fn clear_shrink(&mut self, batch: &RecordBatch); /// 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: + /// https://github.com/apache/datafusion/issues/7065 + /// fn supports_blocked_groups(&self) -> bool { false } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 0706050b73d2..672fb70bacae 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -360,11 +360,16 @@ impl SkipAggregationProbe { /// (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. -/// For a nice introduction to the blocked approach, maybe you can see [#7065]. +/// +/// 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 /// [`maybe_enable_blocked_groups`]. /// +/// [`GroupAccumulator`] /// [`group_values`]: Self::group_values /// [`accumulators`]: Self::accumulators /// [#7065]: https://github.com/apache/datafusion/issues/7065 diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index cb2665965fcc..be9b43ec3457 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -676,14 +676,16 @@ DROP VIEW test.xyz # show_external_create_table() -statement error DataFusion error: Object Store error: Object at location /Users/kamille/Desktop/github/datafusion/testing/data/csv/aggregate_test_100\.csv not found: No such file or directory \(os error 2\) +statement ok CREATE EXTERNAL TABLE abc STORED AS CSV LOCATION '../../testing/data/csv/aggregate_test_100.csv' OPTIONS ('format.has_header' 'true'); -query error DataFusion error: Error during planning: table 'datafusion\.public\.abc' not found +query TTTT SHOW CREATE TABLE abc; +---- +datafusion public abc CREATE EXTERNAL TABLE abc STORED AS CSV LOCATION ../../testing/data/csv/aggregate_test_100.csv # string_agg has different arg_types but same return type. Test avoiding duplicate entries for the same function. query TTT diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 0a844b57ba7a..b2f1b79b3583 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -97,8 +97,8 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.skip_partial_aggregation_probe_rows_threshold | 100000 | Number of input rows partial aggregation partition should process, before aggregation ratio check and trying to switch to skipping aggregation mode | | 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.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.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.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. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | From 948c4ce4518ceba3775dd302b30e0182fc651023 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 02:31:52 +0800 Subject: [PATCH 39/55] move group index operations to a new module. --- .../src/aggregate/groups_accumulator.rs | 77 +--------------- .../groups_accumulator/accumulate.rs | 18 ++-- .../group_index_operations.rs | 91 +++++++++++++++++++ .../group_values/single_group_by/primitive.rs | 4 +- 4 files changed, 103 insertions(+), 87 deletions(-) create mode 100644 datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/group_index_operations.rs diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index af210b587378..453a209051c7 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -20,11 +20,11 @@ pub mod accumulate; pub mod bool_op; +pub mod group_index_operations; pub mod nulls; pub mod prim_op; use std::collections::VecDeque; -use std::fmt::Debug; use std::mem::{size_of, size_of_val}; use arrow::array::new_empty_array; @@ -510,81 +510,6 @@ pub(crate) fn slice_and_maybe_filter( } } -// =============================================== -// Useful tools for group index -// =============================================== - -/// 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 - } -} - // =============================================== // Useful tools for block // =============================================== 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 163411276b3f..850a75b55cbe 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -29,10 +29,10 @@ use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; -use crate::aggregate::groups_accumulator::{ - ensure_room_enough_for_blocks, Block, BlockedGroupIndexOperations, - FlatGroupIndexOperations, GroupIndexOperations, +use crate::aggregate::groups_accumulator::group_index_operations::{ + BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, }; +use crate::aggregate::groups_accumulator::{ensure_room_enough_for_blocks, Block}; /// Track the accumulator null state per row: if any values for that /// group were null and if any values have been seen at all for that group. @@ -245,10 +245,10 @@ pub trait SeenValues: Default + Debug + Send { } /// [`SeenValues`] 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 @@ -264,9 +264,9 @@ pub trait SeenValues: Default + Debug + Send { /// /// 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 -/// +/// #[derive(Debug)] pub struct FlatSeenValues { builder: BooleanBufferBuilder, @@ -325,7 +325,7 @@ impl SeenValues for FlatSeenValues { /// /// 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 @@ -341,7 +341,7 @@ impl SeenValues for FlatSeenValues { /// will be set to `true`. /// /// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator::supports_blocked_groups -/// +/// #[derive(Debug, Default)] pub struct BlockedSeenValues { blocked_builders: VecDeque, 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 000000000000..a62b21fa1be1 --- /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/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 307a149afa25..30ddca201af7 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 @@ -27,7 +27,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::group_index_operations::{ BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, }; use half::f16; @@ -419,7 +419,7 @@ mod tests { use arrow::array::{AsArray, UInt32Array}; use arrow::datatypes::{DataType, UInt32Type}; use datafusion_expr::EmitTo; - use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + use datafusion_functions_aggregate_common::aggregate::groups_accumulator::group_index_operations::{ BlockedGroupIndexOperations, GroupIndexOperations, }; From 09b97abf33363353702f0975036ae9cb49fbe7ca Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 13:59:34 +0800 Subject: [PATCH 40/55] define `Blocks` and use it to refactor. --- .../src/aggregate/groups_accumulator.rs | 125 ++++++++++- .../groups_accumulator/accumulate.rs | 206 +++++------------- .../aggregate/groups_accumulator/prim_op.rs | 57 ++--- 3 files changed, 207 insertions(+), 181 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 453a209051c7..dd2db93f5b5d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -25,9 +25,12 @@ pub mod nulls; pub mod prim_op; use std::collections::VecDeque; +use std::fmt::Debug; use std::mem::{size_of, size_of_val}; +use std::ops::{Index, IndexMut}; +use std::usize; -use arrow::array::new_empty_array; +use arrow::array::{new_empty_array, ArrowNativeTypeOp}; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, compute, @@ -581,10 +584,126 @@ pub(crate) fn ensure_room_enough_for_blocks( } } -pub(crate) trait Block { +pub(crate) trait Block: Debug { type T: Clone; + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); + fn len(&self) -> usize; - fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); + fn size(&self) -> usize; +} + +#[derive(Debug)] +struct Blocks { + inner: VecDeque, + block_size: Option, +} + +impl Blocks { + pub fn new(block_size: Option) -> Self { + Self { + inner: VecDeque::new(), + 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 (mut cur_blk_idx, exist_slots) = if !self.inner.is_empty() { + let cur_blk_idx = self.inner.len() - 1; + let exist_slots = + (self.inner.len() - 1) * block_size + self.inner.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + + // No new groups, don't need to expand, just return + if exist_slots >= total_num_groups { + return; + } + + // 1. Ensure blks are enough + let exist_blks = self.inner.len(); + let new_blks = total_num_groups.div_ceil(block_size) - exist_blks; + if new_blks > 0 { + for _ in 0..new_blks { + let block = new_block(self.block_size); + self.inner.push_back(block); + } + } + + // 2. Ensure slots are enough + let mut new_slots = total_num_groups - exist_slots; + + // 2.1 Only fill current blk if it may be already enough + let cur_blk_rest_slots = block_size - self.inner[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + self.inner[cur_blk_idx].fill_default_value(new_slots, default_value.clone()); + return; + } + + // 2.2 Fill current blk to full + self.inner[cur_blk_idx] + .fill_default_value(cur_blk_rest_slots, default_value.clone()); + new_slots -= cur_blk_rest_slots; + + // 2.3 Fill complete blks + let complete_blks = new_slots / block_size; + for _ in 0..complete_blks { + 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 = new_slots % block_size; + if rest_slots > 0 { + self.inner + .back_mut() + .unwrap() + .fill_default_value(rest_slots, default_value); + } + } + + fn emit_block(&mut self) -> Option { + self.inner.pop_front() + } + + pub fn len(&self) -> usize { + self.inner.len() + } + + pub fn size(&self) -> usize { + self.inner.iter().map(|b| b.size()).sum::() + } + + pub fn clear(&mut self) { + self.inner.clear(); + } +} + +impl Index for Blocks { + type Output = B; + + fn index(&self, index: usize) -> &Self::Output { + &self.inner[index] + } +} + +impl IndexMut for Blocks { + fn index_mut(&mut self, index: usize) -> &mut Self::Output { + &mut self.inner[index] + } } 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 850a75b55cbe..47ace5603fc1 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -32,7 +32,9 @@ use datafusion_expr_common::groups_accumulator::EmitTo; use crate::aggregate::groups_accumulator::group_index_operations::{ BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, }; -use crate::aggregate::groups_accumulator::{ensure_room_enough_for_blocks, Block}; +use crate::aggregate::groups_accumulator::{ + ensure_room_enough_for_blocks, Block, Blocks, +}; /// Track the accumulator null state per row: if any values for that /// group were null and if any values have been seen at all for that group. @@ -60,7 +62,7 @@ use crate::aggregate::groups_accumulator::{ensure_room_enough_for_blocks, Block} /// /// [`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[i]` is true, have seen at least one non null @@ -68,17 +70,17 @@ pub struct NullState { /// /// If `seen_values[i]` is false, have not seen any values that /// pass the filter yet for group `i` - seen_values: V, + seen_values: Blocks, /// phantom data for required type `` _phantom: PhantomData, } -impl NullState { +impl NullState { /// return the size of all buffers allocated by this null state, not including self pub fn size(&self) -> usize { // capacity is in bits, so convert to bytes - self.seen_values.capacity() / 8 + self.seen_values.size() / 8 } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -110,7 +112,10 @@ impl NullState { { // ensure the seen_values is big enough (start everything at // "not seen" valid) - self.seen_values.resize(total_num_groups, false); + let new_block = |block_size: Option| { + BooleanBufferBuilder::new(block_size.unwrap_or(0)) + }; + self.seen_values.resize(total_num_groups, new_block, false); let seen_values = &mut self.seen_values; accumulate(group_indices, values, opt_filter, |packed_index, value| { @@ -147,7 +152,10 @@ impl NullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - self.seen_values.resize(total_num_groups, false); + let new_block = |block_size: Option| { + BooleanBufferBuilder::new(block_size.unwrap_or(0)) + }; + self.seen_values.resize(total_num_groups, new_block, false); // These could be made more performant by iterating in chunks of 64 bits at a time let seen_values = &mut self.seen_values; @@ -233,94 +241,6 @@ impl NullState { } } -/// Structure marking if accumulating groups are seen at least one -pub trait SeenValues: Default + Debug + Send { - fn resize(&mut self, total_num_groups: usize, default_value: bool); - - fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool); - - fn emit(&mut self, emit_to: EmitTo) -> NullBuffer; - - fn capacity(&self) -> usize; -} - -/// [`SeenValues`] 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 -/// -#[derive(Debug)] -pub struct FlatSeenValues { - builder: BooleanBufferBuilder, -} - -impl Default for FlatSeenValues { - fn default() -> Self { - Self { - builder: BooleanBufferBuilder::new(0), - } - } -} - -impl SeenValues for FlatSeenValues { - fn resize(&mut self, total_num_groups: usize, default_value: bool) { - if self.builder.len() < total_num_groups { - let new_groups = total_num_groups - self.builder.len(); - self.builder.append_n(new_groups, default_value); - } - } - - fn set_bit(&mut self, _block_id: u32, block_offset: u64, value: bool) { - self.builder.set_bit(block_offset as usize, value); - } - - fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { - let nulls = self.builder.finish(); - - let nulls = match emit_to { - EmitTo::All => nulls, - EmitTo::First(n) => { - // 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.builder.append(seen); - } - first_n_null - } - EmitTo::NextBlock => { - unreachable!("not support block emit in flat seen values") - } - }; - NullBuffer::new(nulls) - } - - fn capacity(&self) -> usize { - self.builder.capacity() - } -} - /// [`SeenValues`] for `blocked groups input` /// /// At first, you may need to see something about `block_id` and `block_offset` @@ -342,71 +262,55 @@ impl SeenValues for FlatSeenValues { /// /// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator::supports_blocked_groups /// -#[derive(Debug, Default)] -pub struct BlockedSeenValues { - blocked_builders: VecDeque, - - block_size: usize, -} - -impl BlockedSeenValues { - pub fn new(block_size: usize) -> Self { - Self { - blocked_builders: VecDeque::new(), - block_size, - } - } -} - impl Block for BooleanBufferBuilder { type T = bool; - fn len(&self) -> usize { - self.len() - } - fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { self.append_n(fill_len, default_value); } -} -impl SeenValues for BlockedSeenValues { - fn resize(&mut self, total_num_groups: usize, default_value: bool) { - let block_size = self.block_size; - let blocked_builders = &mut self.blocked_builders; - let new_block = |block_size: usize| BooleanBufferBuilder::new(block_size); - - ensure_room_enough_for_blocks( - blocked_builders, - total_num_groups, - block_size, - new_block, - default_value, - ); + fn size(&self) -> usize { + self.capacity() } + fn len(&self) -> usize { + self.len() + } +} + +impl Blocks { fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool) { - self.blocked_builders[block_id as usize].set_bit(block_offset as usize, value); + self[block_id as usize].set_bit(block_offset as usize, value); } fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { - assert!(matches!(emit_to, EmitTo::NextBlock)); + let nulls = match emit_to { + EmitTo::All | EmitTo::First(_) => self[0].finish(), + EmitTo::NextBlock => { + let mut block = self + .emit_block() + .expect("should not try to emit empty blocks"); + block.finish() + } + }; - let mut block = self - .blocked_builders - .pop_front() - .expect("should not try to emit empty blocks"); - let nulls = 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) } - - fn capacity(&self) -> usize { - self.blocked_builders - .iter() - .map(|builder| builder.capacity()) - .sum() - } } /// Adapter for supporting dynamic dispatching of [`FlatNullState`] and [`BlockedNullState`]. @@ -503,11 +407,13 @@ impl NullStateAdapter { fn build_cloned_seen_values(&self) -> BooleanBuffer { match self { NullStateAdapter::Flat(null_state) => { - null_state.seen_values.builder.finish_cloned() + null_state.seen_values[0].finish_cloned() } NullStateAdapter::Blocked(null_state) => { let mut return_builder = BooleanBufferBuilder::new(0); - for builder in &null_state.seen_values.blocked_builders { + let num_blocks = null_state.seen_values.len(); + for blk_idx in 0..num_blocks { + let builder = &null_state.seen_values[blk_idx]; for idx in 0..builder.len() { return_builder.append(builder.get_bit(idx)); } @@ -523,7 +429,7 @@ impl NullStateAdapter { 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.seen_values.blocked_builders.len(); + let num_blocks = null_state.seen_values.len(); for _ in 0..num_blocks { let blocked_nulls = null_state.build(EmitTo::NextBlock); for bit in blocked_nulls.inner().iter() { @@ -538,7 +444,7 @@ impl NullStateAdapter { } /// [`NullState`] implementation for `flat group index` -pub type FlatNullState = NullState; +pub type FlatNullState = NullState; impl FlatNullState { pub fn new() -> Self { @@ -549,19 +455,19 @@ impl FlatNullState { impl Default for FlatNullState { fn default() -> Self { Self { - seen_values: FlatSeenValues::default(), + seen_values: Blocks::new(None), _phantom: PhantomData {}, } } } /// [`NullState`] implementation for `blocked group index` -pub type BlockedNullState = NullState; +pub type BlockedNullState = NullState; impl BlockedNullState { pub fn new(block_size: usize) -> Self { Self { - seen_values: BlockedSeenValues::new(block_size), + seen_values: Blocks::new(Some(block_size)), _phantom: PhantomData {}, } } 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 dc92408e6dd8..d75e46b7243e 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 @@ -29,7 +29,9 @@ use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::NullStateAdapter; -use crate::aggregate::groups_accumulator::{ensure_room_enough_for_blocks, Block}; +use crate::aggregate::groups_accumulator::{ + ensure_room_enough_for_blocks, Block, Blocks, +}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -47,7 +49,7 @@ where F: Fn(&mut T::Native, T::Native) + Send + Sync, { /// Values per group, stored as the native type - values: VecDeque>, + values: Blocks>, /// The output type (needed for Decimal precision and scale) data_type: DataType, @@ -79,7 +81,7 @@ where { pub fn new(data_type: &DataType, prim_fn: F) -> Self { Self { - values: VecDeque::new(), + values: Blocks::new(None), data_type: data_type.clone(), null_state: NullStateAdapter::new(None), starting_value: T::default_value(), @@ -107,31 +109,18 @@ where opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { + const DEFAULT_BLOCK_CAP: usize = 128; + assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); // Expand to ensure values are large enough - if let Some(blk_size) = self.block_size { - // Expand blocks in `blocked mode` - let new_block = |block_size: usize| Vec::with_capacity(block_size); - ensure_room_enough_for_blocks( - &mut self.values, - total_num_groups, - blk_size, - new_block, - self.starting_value, - ); - } else { - // Expand the single block in `flat mode` - if self.values.is_empty() { - self.values.push_back(Vec::new()); - } - - self.values - .back_mut() - .unwrap() - .resize(total_num_groups, self.starting_value); - } + let new_block = |block_size: Option| { + let cap = block_size.unwrap_or(DEFAULT_BLOCK_CAP); + Vec::with_capacity(cap) + }; + 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( @@ -149,7 +138,15 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = emit_to.take_needed(&mut self.values, self.block_size.is_some()); + let values = match emit_to { + EmitTo::All | EmitTo::First(_) => { + emit_to.take_needed_rows(&mut self.values[0]) + } + EmitTo::NextBlock => { + self.values.emit_block().expect("should not call emit for empty blocks") + } + }; + let nulls = self.null_state.build(emit_to); let values = PrimitiveArray::::new(values.into(), Some(nulls)) // no copy .with_data_type(self.data_type.clone()); @@ -230,7 +227,7 @@ where } fn size(&self) -> usize { - self.values.capacity() * size_of::() + self.null_state.size() + self.values.size() + self.null_state.size() } fn supports_blocked_groups(&self) -> bool { @@ -249,11 +246,15 @@ where impl Block for Vec { type T = N; + 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() } - fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { - self.extend(iter::repeat_n(default_value, fill_len)); + fn size(&self) -> usize { + self.capacity() * size_of::() } } From cee016c098735d00d0db91c3f2c315a4d0925c53 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 4 May 2025 23:35:17 +0800 Subject: [PATCH 41/55] extract blocks to a dedicated module. --- .../src/aggregate/groups_accumulator.rs | 198 +----------------- .../groups_accumulator/accumulate.rs | 174 ++++++++------- .../aggregate/groups_accumulator/blocks.rs | 158 ++++++++++++++ .../aggregate/groups_accumulator/prim_op.rs | 13 +- 4 files changed, 261 insertions(+), 282 deletions(-) create mode 100644 datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index dd2db93f5b5d..501d94e1e692 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -28,9 +28,8 @@ use std::collections::VecDeque; use std::fmt::Debug; use std::mem::{size_of, size_of_val}; use std::ops::{Index, IndexMut}; -use std::usize; -use arrow::array::{new_empty_array, ArrowNativeTypeOp}; +use arrow::array::new_empty_array; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, compute, @@ -512,198 +511,3 @@ pub(crate) fn slice_and_maybe_filter( Ok(sliced_arrays) } } - -// =============================================== -// Useful tools for block -// =============================================== -pub(crate) fn ensure_room_enough_for_blocks( - blocks: &mut VecDeque, - total_num_groups: usize, - block_size: usize, - new_block: F, - default_value: B::T, -) where - B: Block, - F: Fn(usize) -> B, -{ - // For resize, we need to: - // 1. Ensure the blks are enough first - // 2. and then ensure slots in blks are enough - let (mut cur_blk_idx, exist_slots) = if !blocks.is_empty() { - let cur_blk_idx = blocks.len() - 1; - let exist_slots = (blocks.len() - 1) * block_size + blocks.back().unwrap().len(); - - (cur_blk_idx, exist_slots) - } else { - (0, 0) - }; - - // No new groups, don't need to expand, just return - if exist_slots >= total_num_groups { - return; - } - - // 1. Ensure blks are enough - let exist_blks = blocks.len(); - let new_blks = total_num_groups.div_ceil(block_size) - exist_blks; - if new_blks > 0 { - for _ in 0..new_blks { - let block = new_block(block_size); - blocks.push_back(block); - } - } - - // 2. Ensure slots are enough - let mut new_slots = total_num_groups - exist_slots; - - // 2.1 Only fill current blk if it may be already enough - let cur_blk_rest_slots = block_size - blocks[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - blocks[cur_blk_idx].fill_default_value(new_slots, default_value.clone()); - return; - } - - // 2.2 Fill current blk to full - blocks[cur_blk_idx].fill_default_value(cur_blk_rest_slots, default_value.clone()); - new_slots -= cur_blk_rest_slots; - - // 2.3 Fill complete blks - let complete_blks = new_slots / block_size; - for _ in 0..complete_blks { - cur_blk_idx += 1; - blocks[cur_blk_idx].fill_default_value(block_size, default_value.clone()); - } - - // 2.4 Fill last blk if needed - let rest_slots = new_slots % block_size; - if rest_slots > 0 { - blocks - .back_mut() - .unwrap() - .fill_default_value(rest_slots, default_value); - } -} - -pub(crate) trait Block: Debug { - type T: Clone; - - fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); - - fn len(&self) -> usize; - - fn size(&self) -> usize; -} - -#[derive(Debug)] -struct Blocks { - inner: VecDeque, - block_size: Option, -} - -impl Blocks { - pub fn new(block_size: Option) -> Self { - Self { - inner: VecDeque::new(), - 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 (mut cur_blk_idx, exist_slots) = if !self.inner.is_empty() { - let cur_blk_idx = self.inner.len() - 1; - let exist_slots = - (self.inner.len() - 1) * block_size + self.inner.back().unwrap().len(); - - (cur_blk_idx, exist_slots) - } else { - (0, 0) - }; - - // No new groups, don't need to expand, just return - if exist_slots >= total_num_groups { - return; - } - - // 1. Ensure blks are enough - let exist_blks = self.inner.len(); - let new_blks = total_num_groups.div_ceil(block_size) - exist_blks; - if new_blks > 0 { - for _ in 0..new_blks { - let block = new_block(self.block_size); - self.inner.push_back(block); - } - } - - // 2. Ensure slots are enough - let mut new_slots = total_num_groups - exist_slots; - - // 2.1 Only fill current blk if it may be already enough - let cur_blk_rest_slots = block_size - self.inner[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - self.inner[cur_blk_idx].fill_default_value(new_slots, default_value.clone()); - return; - } - - // 2.2 Fill current blk to full - self.inner[cur_blk_idx] - .fill_default_value(cur_blk_rest_slots, default_value.clone()); - new_slots -= cur_blk_rest_slots; - - // 2.3 Fill complete blks - let complete_blks = new_slots / block_size; - for _ in 0..complete_blks { - 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 = new_slots % block_size; - if rest_slots > 0 { - self.inner - .back_mut() - .unwrap() - .fill_default_value(rest_slots, default_value); - } - } - - fn emit_block(&mut self) -> Option { - self.inner.pop_front() - } - - pub fn len(&self) -> usize { - self.inner.len() - } - - pub fn size(&self) -> usize { - self.inner.iter().map(|b| b.size()).sum::() - } - - pub fn clear(&mut self) { - self.inner.clear(); - } -} - -impl Index for Blocks { - type Output = B; - - fn index(&self, index: usize) -> &Self::Output { - &self.inner[index] - } -} - -impl IndexMut for Blocks { - fn index_mut(&mut self, index: usize) -> &mut Self::Output { - &mut self.inner[index] - } -} 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 47ace5603fc1..53026865043b 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,7 +19,6 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator -use std::collections::VecDeque; use std::fmt::Debug; use std::marker::PhantomData; @@ -29,12 +28,10 @@ use arrow::datatypes::ArrowPrimitiveType; 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, }; -use crate::aggregate::groups_accumulator::{ - ensure_room_enough_for_blocks, Block, Blocks, -}; /// Track the accumulator null state per row: if any values for that /// group were null and if any values have been seen at all for that group. @@ -241,78 +238,6 @@ impl NullState { } } -/// [`SeenValues`] 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 -/// -impl Block for BooleanBufferBuilder { - type T = bool; - - fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { - self.append_n(fill_len, default_value); - } - - fn size(&self) -> usize { - self.capacity() - } - - fn len(&self) -> usize { - self.len() - } -} - -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 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 - .emit_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) - } -} - /// Adapter for supporting dynamic dispatching of [`FlatNullState`] and [`BlockedNullState`]. /// For performance, the cost of batch-level dynamic dispatching is acceptable. #[derive(Debug)] @@ -443,7 +368,29 @@ impl NullStateAdapter { } } -/// [`NullState`] implementation for `flat group index` +/// [`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; impl FlatNullState { @@ -461,7 +408,27 @@ impl Default for FlatNullState { } } -/// [`NullState`] implementation for `blocked group index` +/// [`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 BlockedNullState { @@ -473,6 +440,57 @@ impl BlockedNullState { } } +impl Block for BooleanBufferBuilder { + type T = bool; + + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { + self.append_n(fill_len, default_value); + } + + fn len(&self) -> usize { + self.len() + } + + fn size(&self) -> usize { + self.capacity() + } +} + +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 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 /// filtered value of `value`, /// 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 000000000000..31ba855d9139 --- /dev/null +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -0,0 +1,158 @@ +// 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 + +/// 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. +/// +/// 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: VecDeque, + block_size: Option, +} + +impl Blocks { + pub fn new(block_size: Option) -> Self { + Self { + inner: VecDeque::new(), + 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 (mut cur_blk_idx, exist_slots) = if !self.inner.is_empty() { + let cur_blk_idx = self.inner.len() - 1; + let exist_slots = + (self.inner.len() - 1) * block_size + self.inner.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + + // No new groups, don't need to expand, just return + if exist_slots >= total_num_groups { + return; + } + + // 1. Ensure blks are enough + let exist_blks = self.inner.len(); + let new_blks = total_num_groups.div_ceil(block_size) - exist_blks; + if new_blks > 0 { + for _ in 0..new_blks { + let block = new_block(self.block_size); + self.inner.push_back(block); + } + } + + // 2. Ensure slots are enough + let mut new_slots = total_num_groups - exist_slots; + + // 2.1 Only fill current blk if it may be already enough + let cur_blk_rest_slots = block_size - self.inner[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + self.inner[cur_blk_idx].fill_default_value(new_slots, default_value.clone()); + return; + } + + // 2.2 Fill current blk to full + self.inner[cur_blk_idx] + .fill_default_value(cur_blk_rest_slots, default_value.clone()); + new_slots -= cur_blk_rest_slots; + + // 2.3 Fill complete blks + let complete_blks = new_slots / block_size; + for _ in 0..complete_blks { + 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 = new_slots % block_size; + if rest_slots > 0 { + self.inner + .back_mut() + .unwrap() + .fill_default_value(rest_slots, default_value); + } + } + + pub fn pop_block(&mut self) -> Option { + self.inner.pop_front() + } + + pub fn len(&self) -> usize { + self.inner.len() + } + + pub fn size(&self) -> usize { + self.inner.iter().map(|b| b.size()).sum::() + } + + pub fn clear(&mut self) { + self.inner.clear(); + } +} + +impl Index for Blocks { + type Output = B; + + fn index(&self, index: usize) -> &Self::Output { + &self.inner[index] + } +} + +impl IndexMut for Blocks { + fn index_mut(&mut self, index: usize) -> &mut Self::Output { + &mut self.inner[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 { + type T: Clone; + + fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); + + fn len(&self) -> usize; + + fn size(&self) -> usize; +} 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 d75e46b7243e..19fadc002727 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 @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::collections::VecDeque; use std::iter; use std::mem::size_of; use std::sync::Arc; @@ -29,9 +28,7 @@ use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::NullStateAdapter; -use crate::aggregate::groups_accumulator::{ - ensure_room_enough_for_blocks, Block, Blocks, -}; +use crate::aggregate::groups_accumulator::blocks::{Block, Blocks}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -142,9 +139,10 @@ where EmitTo::All | EmitTo::First(_) => { emit_to.take_needed_rows(&mut self.values[0]) } - EmitTo::NextBlock => { - self.values.emit_block().expect("should not call emit for empty blocks") - } + EmitTo::NextBlock => self + .values + .pop_block() + .expect("should not call emit for empty blocks"), }; let nulls = self.null_state.build(emit_to); @@ -236,6 +234,7 @@ where 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); self.block_size = block_size; From 75ee3f371a004330a14401a820d07fa60123edc8 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 5 May 2025 01:23:36 +0800 Subject: [PATCH 42/55] add tests for `Blocks`. --- .../src/aggregate/groups_accumulator.rs | 4 +- .../aggregate/groups_accumulator/blocks.rs | 109 +++++++++++++++++- 2 files changed, 109 insertions(+), 4 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 501d94e1e692..b2c41a00cc7f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -19,15 +19,13 @@ //! 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; -use std::collections::VecDeque; -use std::fmt::Debug; use std::mem::{size_of, size_of_val}; -use std::ops::{Index, IndexMut}; use arrow::array::new_empty_array; use arrow::{ diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs index 31ba855d9139..ff0d24dcb76d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -17,6 +17,12 @@ //! Aggregation intermediate results blocks in blocked approach +use std::{ + collections::VecDeque, + fmt, + ops::{Index, IndexMut}, +}; + /// Structure used to store aggregation intermediate results in `blocked approach` /// /// Aggregation intermediate results will be stored as multiple [`Block`]s @@ -118,6 +124,10 @@ impl Blocks { self.inner.len() } + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + pub fn size(&self) -> usize { self.inner.iter().map(|b| b.size()).sum::() } @@ -147,12 +157,109 @@ impl IndexMut for 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 { +pub trait Block: fmt::Debug { type T: Clone; fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); fn len(&self) -> usize; + fn is_empty(&self) -> bool { + self.len() == 0 + } + fn size(&self) -> usize; } + +#[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.len(), 0); + assert_eq!(blocks.size(), 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.len(), 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.len(), 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.len(), 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.len(), 0); + assert_eq!(blocks.size(), 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.len(), 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.len(), 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.len(), 0); + + // Test resize after clear in next round + } + } +} From 5a6e0301858f769921a387ba0149d4143d7b1c79 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 5 May 2025 13:01:15 +0800 Subject: [PATCH 43/55] simplify codes about blocks. --- datafusion-examples/examples/advanced_udaf.rs | 8 +-- .../expr-common/src/groups_accumulator.rs | 39 +------------- .../src/aggregate/groups_accumulator.rs | 4 +- .../groups_accumulator/accumulate.rs | 8 +-- .../aggregate/groups_accumulator/blocks.rs | 54 ++++++++++++++++--- .../aggregate/groups_accumulator/prim_op.rs | 50 +++-------------- datafusion/functions-aggregate/src/average.rs | 8 +-- datafusion/functions-aggregate/src/count.rs | 4 +- .../functions-aggregate/src/first_last.rs | 6 +-- datafusion/functions-aggregate/src/median.rs | 4 +- .../functions-aggregate/src/variance.rs | 12 ++--- 11 files changed, 82 insertions(+), 115 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 727be7017491..875841a01a62 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -314,8 +314,8 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { /// Generate output, as specified by `emit_to` and update the intermediate state fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed_rows(&mut self.counts); - let prods = emit_to.take_needed_rows(&mut self.prods); + let counts = emit_to.take_needed(&mut self.counts); + let prods = emit_to.take_needed(&mut self.prods); let nulls = self.null_state.build(emit_to); assert_eq!(nulls.len(), prods.len()); @@ -353,10 +353,10 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); - let counts = emit_to.take_needed_rows(&mut self.counts); + let counts = emit_to.take_needed(&mut self.counts); let counts = UInt32Array::new(counts.into(), nulls.clone()); // zero copy - let prods = emit_to.take_needed_rows(&mut self.prods); + let prods = emit_to.take_needed(&mut self.prods); let prods = PrimitiveArray::::new(prods.into(), nulls) // zero copy .with_data_type(self.prod_data_type.clone()); diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 508dd087df2d..6873e9e53da5 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,7 +17,6 @@ //! Vectorized [`GroupsAccumulator`] -use std::collections::VecDeque; use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -40,28 +39,6 @@ pub enum EmitTo { } impl EmitTo { - /// Remove and return `needed values` from `values`. - /// - /// Inputs: - /// - `values`, the emitting source. - /// - `is_blocked_groups`, is the `values` organized in `single` - /// or `blocked` approach, more details can see - /// [`GroupsAccumulator::supports_blocked_groups`]. - /// - /// - pub fn take_needed( - &self, - values: &mut VecDeque>, - is_blocked_groups: bool, - ) -> Vec { - if is_blocked_groups { - self.take_needed_block(values) - } else { - assert_eq!(values.len(), 1); - self.take_needed_rows(values.back_mut().unwrap()) - } - } - /// Removes the number of rows from `v` required to emit the right /// number of rows, returning a `Vec` with elements taken, and the /// remaining values in `v`. @@ -70,7 +47,7 @@ impl EmitTo { /// /// NOTICE: only support emit strategies: `Self::All` and `Self::First` /// - pub fn take_needed_rows(&self, v: &mut Vec) -> Vec { + pub fn take_needed(&self, v: &mut Vec) -> Vec { match self { Self::All => { // Take the entire vector, leave new (empty) vector @@ -86,20 +63,6 @@ impl EmitTo { Self::NextBlock => unreachable!("don't support take block in take_needed"), } } - - /// Removes one block required to emit and return it - /// - /// NOTICE: only support emit strategy `Self::NextBlock` - /// - fn take_needed_block(&self, blocks: &mut VecDeque>) -> Vec { - assert!( - matches!(self, Self::NextBlock), - "only support take block in take_needed_block" - ); - blocks - .pop_front() - .expect("should not call emit for empty blocks") - } } /// `GroupsAccumulator` implements a single aggregate (e.g. AVG) and diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index b2c41a00cc7f..77c4a89b2ac5 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -323,7 +323,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { fn evaluate(&mut self, emit_to: EmitTo) -> Result { let vec_size_pre = self.states.allocated_size(); - let states = emit_to.take_needed_rows(&mut self.states); + let states = emit_to.take_needed(&mut self.states); let results: Vec = states .into_iter() @@ -343,7 +343,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { // filtered_null_mask(opt_filter, &values); fn state(&mut self, emit_to: EmitTo) -> Result> { let vec_size_pre = self.states.allocated_size(); - let states = emit_to.take_needed_rows(&mut self.states); + let states = emit_to.take_needed(&mut self.states); // each accumulator produces a potential vector of values // which we need to form into columns 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 53026865043b..b89fc4c36f80 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -450,10 +450,6 @@ impl Block for BooleanBufferBuilder { fn len(&self) -> usize { self.len() } - - fn size(&self) -> usize { - self.capacity() - } } impl Blocks { @@ -489,6 +485,10 @@ impl Blocks { NullBuffer::new(nulls) } + + pub fn size(&self) -> usize { + self.iter().map(|b| b.capacity()).sum::() + } } /// Invokes `value_fn(group_index, value)` for each non null, non diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs index ff0d24dcb76d..28dfb31ca3fa 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -19,10 +19,13 @@ use std::{ collections::VecDeque, - fmt, + fmt::Debug, + iter, 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 @@ -128,8 +131,8 @@ impl Blocks { self.inner.is_empty() } - pub fn size(&self) -> usize { - self.inner.iter().map(|b| b.size()).sum::() + pub fn iter(&self) -> impl Iterator { + self.inner.iter() } pub fn clear(&mut self) { @@ -157,7 +160,7 @@ impl IndexMut for 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: fmt::Debug { +pub trait Block: Debug { type T: Clone; fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); @@ -167,8 +170,47 @@ pub trait Block: fmt::Debug { fn is_empty(&self) -> bool { self.len() == 0 } +} + +/// Usually we use `Vec` to represent `Block`, so we define `Blocks>` +/// as the `GeneralBlocks` +pub type GeneralBlocks = Blocks>; + +/// As mentioned in [`GeneralBlocks`], we usually use `Vec` to represent `Block`, +/// so we implement `Block` trait for `Vec` +impl Block for Vec { + type T = Ty; - fn size(&self) -> usize; + 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 GeneralBlocks { + 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.inner + .pop_front() + .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.inner[0]) + } + } } #[cfg(test)] @@ -186,7 +228,6 @@ mod test { let mut blocks = TestBlocks::new(None); assert_eq!(blocks.len(), 0); - assert_eq!(blocks.size(), 0); for _ in 0..2 { // Should have single block, 5 block len, all data are 42 @@ -220,7 +261,6 @@ mod test { let mut blocks = TestBlocks::new(Some(3)); assert_eq!(blocks.len(), 0); - assert_eq!(blocks.size(), 0); for _ in 0..2 { // Should have: 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 19fadc002727..c8c906800a01 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 @@ -15,11 +15,10 @@ // specific language governing permissions and limitations // under the License. -use std::iter; use std::mem::size_of; use std::sync::Arc; -use arrow::array::{ArrayRef, ArrowNativeTypeOp, AsArray, BooleanArray, PrimitiveArray}; +use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; use arrow::buffer::NullBuffer; use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; @@ -28,7 +27,7 @@ use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::NullStateAdapter; -use crate::aggregate::groups_accumulator::blocks::{Block, Blocks}; +use crate::aggregate::groups_accumulator::blocks::{Blocks, GeneralBlocks}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -46,7 +45,7 @@ where F: Fn(&mut T::Native, T::Native) + Send + Sync, { /// Values per group, stored as the native type - values: Blocks>, + values: GeneralBlocks, /// The output type (needed for Decimal precision and scale) data_type: DataType, @@ -59,16 +58,6 @@ where /// Function that computes the primitive result prim_fn: F, - - /// Block size of current `GroupAccumulator` 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 PrimitiveGroupsAccumulator @@ -83,7 +72,6 @@ where null_state: NullStateAdapter::new(None), starting_value: T::default_value(), prim_fn, - block_size: None, } } @@ -135,16 +123,7 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = match emit_to { - EmitTo::All | EmitTo::First(_) => { - emit_to.take_needed_rows(&mut self.values[0]) - } - EmitTo::NextBlock => self - .values - .pop_block() - .expect("should not call emit for empty blocks"), - }; - + let values = self.values.emit(emit_to); let nulls = self.null_state.build(emit_to); let values = PrimitiveArray::::new(values.into(), Some(nulls)) // no copy .with_data_type(self.data_type.clone()); @@ -225,7 +204,9 @@ where } fn size(&self) -> usize { - self.values.size() + self.null_state.size() + let values_cap = self.values.iter().map(|b| b.capacity()).sum::(); + let values_size = values_cap * size_of::(); + values_size + self.null_state.size() } fn supports_blocked_groups(&self) -> bool { @@ -236,24 +217,7 @@ where self.values.clear(); self.values = Blocks::new(block_size); self.null_state = NullStateAdapter::new(block_size); - self.block_size = block_size; Ok(()) } } - -impl Block for Vec { - type T = N; - - 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() - } - - fn size(&self) -> usize { - self.capacity() * size_of::() - } -} diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 5e7d77cdbba6..809a58097505 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -601,8 +601,8 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed_rows(&mut self.counts); - let sums = emit_to.take_needed_rows(&mut self.sums); + let counts = emit_to.take_needed(&mut self.counts); + let sums = emit_to.take_needed(&mut self.sums); let nulls = self.null_state.build(emit_to); assert_eq!(nulls.len(), sums.len()); @@ -641,10 +641,10 @@ where let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); - let counts = emit_to.take_needed_rows(&mut self.counts); + let counts = emit_to.take_needed(&mut self.counts); let counts = UInt64Array::new(counts.into(), nulls.clone()); // zero copy - let sums = emit_to.take_needed_rows(&mut self.sums); + let sums = emit_to.take_needed(&mut self.sums); let sums = PrimitiveArray::::new(sums.into(), nulls) // zero copy .with_data_type(self.sum_data_type.clone()); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 1b88c9921023..2d995b4a4179 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -537,7 +537,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed_rows(&mut self.counts); + let counts = emit_to.take_needed(&mut self.counts); // Count is always non null (null inputs just don't contribute to the overall values) let nulls = None; @@ -548,7 +548,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { // return arrays for counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let counts = emit_to.take_needed_rows(&mut self.counts); + let counts = emit_to.take_needed(&mut self.counts); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls Ok(vec![Arc::new(counts) as ArrayRef]) } diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index c25da41225a1..79aeb4412115 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -428,7 +428,7 @@ where } fn take_orderings(&mut self, emit_to: EmitTo) -> Vec> { - let result = emit_to.take_needed_rows(&mut self.orderings); + let result = emit_to.take_needed(&mut self.orderings); match emit_to { EmitTo::All => self.size_of_orderings = 0, @@ -519,7 +519,7 @@ where &mut self, emit_to: EmitTo, ) -> (ArrayRef, Vec>, BooleanBuffer) { - emit_to.take_needed_rows(&mut self.min_of_each_group_buf.0); + emit_to.take_needed(&mut self.min_of_each_group_buf.0); self.min_of_each_group_buf .1 .truncate(self.min_of_each_group_buf.0.len()); @@ -621,7 +621,7 @@ where } fn take_vals_and_null_buf(&mut self, emit_to: EmitTo) -> ArrayRef { - let r = emit_to.take_needed_rows(&mut self.vals); + let r = emit_to.take_needed(&mut self.vals); let null_buf = NullBuffer::new(Self::take_need(&mut self.null_builder, emit_to)); diff --git a/datafusion/functions-aggregate/src/median.rs b/datafusion/functions-aggregate/src/median.rs index 32931f56c0ee..ba6b63260e06 100644 --- a/datafusion/functions-aggregate/src/median.rs +++ b/datafusion/functions-aggregate/src/median.rs @@ -393,7 +393,7 @@ impl GroupsAccumulator for MedianGroupsAccumulator Result> { // Emit values - let emit_group_values = emit_to.take_needed_rows(&mut self.group_values); + let emit_group_values = emit_to.take_needed(&mut self.group_values); // Build offsets let mut offsets = Vec::with_capacity(self.group_values.len() + 1); @@ -432,7 +432,7 @@ impl GroupsAccumulator for MedianGroupsAccumulator Result { // Emit values - let emit_group_values = emit_to.take_needed_rows(&mut self.group_values); + let emit_group_values = emit_to.take_needed(&mut self.group_values); // Calculate median for each group let mut evaluate_result_builder = diff --git a/datafusion/functions-aggregate/src/variance.rs b/datafusion/functions-aggregate/src/variance.rs index 5947fbf2c81d..53e3e0cc56cd 100644 --- a/datafusion/functions-aggregate/src/variance.rs +++ b/datafusion/functions-aggregate/src/variance.rs @@ -460,11 +460,11 @@ impl VarianceGroupsAccumulator { &mut self, emit_to: datafusion_expr::EmitTo, ) -> (Vec, NullBuffer) { - let mut counts = emit_to.take_needed_rows(&mut self.counts); + let mut counts = emit_to.take_needed(&mut self.counts); // means are only needed for updating m2s and are not needed for the final result. // But we still need to take them to ensure the internal state is consistent. - let _ = emit_to.take_needed_rows(&mut self.means); - let m2s = emit_to.take_needed_rows(&mut self.m2s); + let _ = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); if let StatsType::Sample = self.stats_type { counts.iter_mut().for_each(|count| { @@ -555,9 +555,9 @@ impl GroupsAccumulator for VarianceGroupsAccumulator { } fn state(&mut self, emit_to: datafusion_expr::EmitTo) -> Result> { - let counts = emit_to.take_needed_rows(&mut self.counts); - let means = emit_to.take_needed_rows(&mut self.means); - let m2s = emit_to.take_needed_rows(&mut self.m2s); + let counts = emit_to.take_needed(&mut self.counts); + let means = emit_to.take_needed(&mut self.means); + let m2s = emit_to.take_needed(&mut self.m2s); Ok(vec![ Arc::new(UInt64Array::new(counts.into(), None)), From 4c6799f2a5a62cbecd66dd03fa3fd92678005022 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 5 May 2025 13:56:06 +0800 Subject: [PATCH 44/55] fix ci. --- datafusion/expr-common/src/groups_accumulator.rs | 3 +-- datafusion/physical-plan/src/aggregates/group_values/mod.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 3 +-- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 6873e9e53da5..53bcd094a205 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,7 +17,6 @@ //! Vectorized [`GroupsAccumulator`] - use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -278,7 +277,7 @@ pub trait GroupsAccumulator: Send { /// to locate data in the single `Block`. /// /// More details can see: - /// https://github.com/apache/datafusion/issues/7065 + /// /// fn supports_blocked_groups(&self) -> bool { false diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index ffe110502ada..e8da762e0f39 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -129,7 +129,7 @@ pub(crate) trait GroupValues: Send { /// to locate data in the single `Block`. /// /// More details can see: - /// https://github.com/apache/datafusion/issues/7065 + /// /// fn supports_blocked_groups(&self) -> bool { false diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 672fb70bacae..d43a5fee3c9e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -369,10 +369,9 @@ impl SkipAggregationProbe { /// The conditions that trigger the blocked groups optimization can be found in /// [`maybe_enable_blocked_groups`]. /// -/// [`GroupAccumulator`] /// [`group_values`]: Self::group_values /// [`accumulators`]: Self::accumulators -/// [#7065]: https://github.com/apache/datafusion/issues/7065 +/// [#7065]: /// pub(crate) struct GroupedHashAggregateStream { // ======================================================================== From 7542b49a2add7acae700de6e88a7cf8227cc6735 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 5 May 2025 14:15:59 +0800 Subject: [PATCH 45/55] return error when found blocked approach not supported. --- .../aggregate/groups_accumulator/bool_op.rs | 4 ++-- .../functions-aggregate/src/correlation.rs | 6 +++--- .../functions-aggregate/src/first_last.rs | 21 ++++++++++--------- .../src/min_max/min_max_bytes.rs | 12 +++++------ .../group_values/multi_group_by/mod.rs | 6 ++++-- .../src/aggregates/group_values/row.rs | 4 ++-- .../group_values/single_group_by/bytes.rs | 5 ++++- .../single_group_by/bytes_view.rs | 5 ++++- 8 files changed, 36 insertions(+), 27 deletions(-) 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 41c32d81b996..c3713489603e 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,7 +20,7 @@ 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::{internal_err, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use super::accumulate::FlatNullState; @@ -119,7 +119,7 @@ where first_n } EmitTo::NextBlock => { - unreachable!("this accumulator still not support blocked groups") + return internal_err!("boolean_op does not support blocked groups") } }; diff --git a/datafusion/functions-aggregate/src/correlation.rs b/datafusion/functions-aggregate/src/correlation.rs index 191458e185a4..61dd06b4028d 100644 --- a/datafusion/functions-aggregate/src/correlation.rs +++ b/datafusion/functions-aggregate/src/correlation.rs @@ -38,7 +38,7 @@ use log::debug; use crate::covariance::CovarianceAccumulator; use crate::stddev::StddevAccumulator; -use datafusion_common::{plan_err, Result, ScalarValue}; +use datafusion_common::{internal_err, plan_err, Result, ScalarValue}; use datafusion_expr::{ function::{AccumulatorArgs, StateFieldsArgs}, type_coercion::aggregates::NUMERICS, @@ -449,7 +449,7 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { EmitTo::All => self.count.len(), EmitTo::First(n) => n, EmitTo::NextBlock => { - unreachable!("this accumulator still not support blocked groups") + return internal_err!("correlation does not support blocked groups") } }; @@ -505,7 +505,7 @@ impl GroupsAccumulator for CorrelationGroupsAccumulator { EmitTo::All => self.count.len(), EmitTo::First(n) => n, EmitTo::NextBlock => { - unreachable!("this accumulator still not support blocked groups") + return internal_err!("correlation does not support blocked groups") } }; diff --git a/datafusion/functions-aggregate/src/first_last.rs b/datafusion/functions-aggregate/src/first_last.rs index 79aeb4412115..d1626b9d6981 100644 --- a/datafusion/functions-aggregate/src/first_last.rs +++ b/datafusion/functions-aggregate/src/first_last.rs @@ -427,21 +427,21 @@ where }) } - 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 => { - unreachable!("this accumulator still not support blocked groups") + return internal_err!("first_last does not support blocked groups") } } - result + Ok(result) } fn take_need( @@ -518,17 +518,18 @@ where fn take_state( &mut self, emit_to: EmitTo, - ) -> (ArrayRef, Vec>, BooleanBuffer) { + ) -> Result<(ArrayRef, Vec>, BooleanBuffer)> { emit_to.take_needed(&mut self.min_of_each_group_buf.0); self.min_of_each_group_buf .1 .truncate(self.min_of_each_group_buf.0.len()); - ( + let orderings = self.take_orderings(emit_to)?; + Ok(( self.take_vals_and_null_buf(emit_to), - self.take_orderings(emit_to), + orderings, Self::take_need(&mut self.is_sets, emit_to), - ) + )) } // should be used in test only @@ -680,11 +681,11 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - Ok(self.take_state(emit_to).0) + self.take_state(emit_to).map(|state| state.0) } fn state(&mut self, emit_to: EmitTo) -> Result> { - let (val_arr, orderings, is_sets) = self.take_state(emit_to); + let (val_arr, orderings, is_sets) = self.take_state(emit_to)?; let mut result = Vec::with_capacity(self.orderings.len() + 2); result.push(val_arr); 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 3615a8b919de..f361ee8ad667 100644 --- a/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs +++ b/datafusion/functions-aggregate/src/min_max/min_max_bytes.rs @@ -199,7 +199,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( @@ -494,13 +494,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(); @@ -509,10 +509,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 => { - unreachable!("this accumulator still not support blocked groups") + internal_err!("min/max bytes does not support blocked groups") } } } 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 fb5f256c52af..4402c898916c 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 @@ -40,7 +40,7 @@ use arrow::datatypes::{ UInt8Type, }; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{HashTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; @@ -1155,7 +1155,9 @@ impl GroupValues for GroupValuesColumn { output } EmitTo::NextBlock => { - unreachable!("this group values still not support blocked groups") + return internal_err!( + "group_values_column does not support blocked groups" + ) } }; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 4a6df775062c..61d57dd37e53 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -22,7 +22,7 @@ use arrow::compute::cast; use arrow::datatypes::{DataType, SchemaRef}; use arrow::row::{RowConverter, Rows, SortField}; use datafusion_common::hash_utils::create_hashes; -use datafusion_common::Result; +use datafusion_common::{internal_err, Result}; use datafusion_execution::memory_pool::proxy::{HashTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use hashbrown::hash_table::HashTable; @@ -231,7 +231,7 @@ impl GroupValues for GroupValuesRows { output } EmitTo::NextBlock => { - unreachable!("this group values still not support blocked groups") + return internal_err!("group_values_rows does not support blocked groups") } }; 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 bdc02bcc34e7..f2f04efbc1cf 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 @@ -17,6 +17,7 @@ use crate::aggregates::group_values::GroupValues; use arrow::array::{Array, ArrayRef, OffsetSizeTrait, RecordBatch}; +use datafusion_common::internal_err; use datafusion_expr::EmitTo; use datafusion_physical_expr_common::binary_map::{ArrowBytesMap, OutputType}; use std::mem::size_of; @@ -117,7 +118,9 @@ impl GroupValues for GroupValuesByes { emit_group_values } EmitTo::NextBlock => { - unreachable!("this group values still not support blocked groups") + return internal_err!( + "group_values_bytes does not support blocked groups" + ) } }; 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 610fea924a05..07ab92f32826 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, RecordBatch}; +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; @@ -118,7 +119,9 @@ impl GroupValues for GroupValuesBytesView { emit_group_values } EmitTo::NextBlock => { - unreachable!("this group values still not support blocked groups") + return internal_err!( + "group_values_bytes_view does not support blocked groups" + ) } }; From e8808eb623f821fc0c463eed5b0e3031ac5e0ea0 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 6 May 2025 10:41:29 +0800 Subject: [PATCH 46/55] try to avoid using of `VecDeque`. --- .../groups_accumulator/accumulate.rs | 33 +++++++----- .../aggregate/groups_accumulator/blocks.rs | 28 ++++++---- .../group_values/single_group_by/primitive.rs | 52 +++++++++---------- 3 files changed, 66 insertions(+), 47 deletions(-) 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 b89fc4c36f80..d0304b9c4507 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -67,7 +67,7 @@ pub struct NullState { /// /// If `seen_values[i]` is false, have not seen any values that /// pass the filter yet for group `i` - seen_values: Blocks, + seen_values: Blocks, /// phantom data for required type `` _phantom: PhantomData, @@ -110,7 +110,7 @@ impl NullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) let new_block = |block_size: Option| { - BooleanBufferBuilder::new(block_size.unwrap_or(0)) + BooleanBufferBuilderWrapper(BooleanBufferBuilder::new(block_size.unwrap_or(0))) }; self.seen_values.resize(total_num_groups, new_block, false); @@ -150,7 +150,7 @@ impl NullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) let new_block = |block_size: Option| { - BooleanBufferBuilder::new(block_size.unwrap_or(0)) + BooleanBufferBuilderWrapper(BooleanBufferBuilder::new(block_size.unwrap_or(0))) }; self.seen_values.resize(total_num_groups, new_block, false); @@ -440,31 +440,40 @@ impl BlockedNullState { } } -impl Block for BooleanBufferBuilder { +impl Block for BooleanBufferBuilderWrapper { type T = bool; fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T) { - self.append_n(fill_len, default_value); + self.0.append_n(fill_len, default_value); } fn len(&self) -> usize { - self.len() + self.0.len() } } -impl Blocks { +#[derive(Debug)] +struct BooleanBufferBuilderWrapper(BooleanBufferBuilder); + +impl Default for BooleanBufferBuilderWrapper { + fn default() -> Self { + Self(BooleanBufferBuilder::new(0)) + } +} + +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); + self[block_id as usize].0.set_bit(block_offset as usize, value); } fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { let nulls = match emit_to { - EmitTo::All | EmitTo::First(_) => self[0].finish(), + EmitTo::All | EmitTo::First(_) => self[0].0.finish(), EmitTo::NextBlock => { let mut block = self .pop_block() .expect("should not try to emit empty blocks"); - block.finish() + block.0.finish() } }; @@ -476,7 +485,7 @@ impl Blocks { 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); + self[0].0.append(seen); } first_n_null } else { @@ -487,7 +496,7 @@ impl Blocks { } pub fn size(&self) -> usize { - self.iter().map(|b| b.capacity()).sum::() + self.iter().map(|b| b.0.capacity()).sum::() } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs index 28dfb31ca3fa..867e0c5cea13 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -38,14 +38,16 @@ use datafusion_expr_common::groups_accumulator::EmitTo; /// #[derive(Debug)] pub struct Blocks { - inner: VecDeque, + inner: Vec, + next_emit_block_id: usize, block_size: Option, } impl Blocks { pub fn new(block_size: Option) -> Self { Self { - inner: VecDeque::new(), + inner: Vec::new(), + next_emit_block_id: 0, block_size, } } @@ -65,7 +67,7 @@ impl Blocks { let (mut cur_blk_idx, exist_slots) = if !self.inner.is_empty() { let cur_blk_idx = self.inner.len() - 1; let exist_slots = - (self.inner.len() - 1) * block_size + self.inner.back().unwrap().len(); + (self.inner.len() - 1) * block_size + self.inner.last().unwrap().len(); (cur_blk_idx, exist_slots) } else { @@ -83,7 +85,7 @@ impl Blocks { if new_blks > 0 { for _ in 0..new_blks { let block = new_block(self.block_size); - self.inner.push_back(block); + self.inner.push(block); } } @@ -113,14 +115,22 @@ impl Blocks { let rest_slots = new_slots % block_size; if rest_slots > 0 { self.inner - .back_mut() + .last_mut() .unwrap() .fill_default_value(rest_slots, default_value); } } pub fn pop_block(&mut self) -> Option { - self.inner.pop_front() + if self.next_emit_block_id >= self.inner.len() { + return None; + } + + let emit_block_id = self.next_emit_block_id; + let emit_blk = std::mem::take(&mut self.inner[emit_block_id]); + self.next_emit_block_id += 1; + + Some(emit_blk) } pub fn len(&self) -> usize { @@ -137,6 +147,7 @@ impl Blocks { pub fn clear(&mut self) { self.inner.clear(); + self.next_emit_block_id = 0; } } @@ -160,7 +171,7 @@ impl IndexMut for 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 { +pub trait Block: Debug + Default { type T: Clone; fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); @@ -197,8 +208,7 @@ impl GeneralBlocks { self.block_size.is_some(), "only support emit next block in blocked groups" ); - self.inner - .pop_front() + self.pop_block() .expect("should not call emit for empty blocks") } else { // TODO: maybe remove `EmitTo::take_needed` and move the 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 30ddca201af7..b9fe42fac974 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 @@ -96,7 +96,9 @@ pub struct GroupValuesPrimitive { null_group: Option, /// The values for each group index - values: VecDeque>, + values: Vec>, + + next_emit_block_id: usize, /// The random state used to generate hashes random_state: RandomState, @@ -118,13 +120,14 @@ impl GroupValuesPrimitive { // 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()); + let mut values = Vec::new(); + values.push(Vec::new()); Self { data_type, map: HashTable::with_capacity(128), values, + next_emit_block_id: 0, null_group: None, random_state: Default::default(), block_size: None, @@ -138,12 +141,12 @@ where { 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>| { + let before_add_group = |group_values: &mut Vec>| { if group_values.is_empty() - || group_values.back().unwrap().len() == block_size + || group_values.last().unwrap().len() == block_size { let new_block = Vec::with_capacity(block_size); - group_values.push_back(new_block); + group_values.push(new_block); } }; self.get_or_create_groups::<_, BlockedGroupIndexOperations>( @@ -155,7 +158,7 @@ where self.get_or_create_groups::<_, FlatGroupIndexOperations>( cols, groups, - |_: &mut VecDeque>| {}, + |_: &mut Vec>| {}, ) } } @@ -205,7 +208,7 @@ where self.map.clear(); build_primitive( - std::mem::take(self.values.back_mut().unwrap()), + std::mem::take(self.values.last_mut().unwrap()), self.null_group.take().map(|idx| idx as usize), ) } @@ -239,7 +242,7 @@ where None => None, }; - let single_block = self.values.back_mut().unwrap(); + let single_block = self.values.last_mut().unwrap(); let mut split = single_block.split_off(n as usize); std::mem::swap(single_block, &mut split); build_primitive(split, null_group.map(|idx| idx as usize)) @@ -258,6 +261,12 @@ where // in `map` and `null_group` self.map.clear(); + // Get current emit block id firstly + let emit_block_id = self.next_emit_block_id; + let emit_blk = std::mem::take(&mut self.values[emit_block_id]); + self.next_emit_block_id += 1; + + // Check if `null` is in current block let null_block_pair_opt = self.null_group.map(|packed_idx| { ( BlockedGroupIndexOperations::get_block_id(packed_idx), @@ -265,22 +274,12 @@ where ) }); 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_idx = BlockedGroupIndexOperations::pack_index( - new_blk_id, blk_offset, - ); - self.null_group = Some(new_packed_idx); - None - } - Some((_, blk_offset)) => { - self.null_group = None; + Some((blk_id, blk_offset)) if blk_id as usize == emit_block_id => { Some(blk_offset as usize) } - None => None, + _ => None, }; - let emit_blk = self.values.pop_front().unwrap(); build_primitive(emit_blk, null_idx) } }; @@ -295,7 +294,7 @@ where // 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(); + let single_block = self.values.last_mut().unwrap(); single_block.clear(); single_block.shrink_to(count); } @@ -313,11 +312,12 @@ where self.values.clear(); self.null_group = None; self.block_size = block_size; + self.next_emit_block_id = 0; // As mentioned above, we ensure the `single block` always exist // in `flat mode` if block_size.is_none() { - self.values.push_back(Vec::new()); + self.values.push(Vec::new()); } Ok(()) @@ -335,7 +335,7 @@ where mut before_add_group: F, ) -> Result<()> where - F: FnMut(&mut VecDeque>), + F: FnMut(&mut Vec>), O: GroupIndexOperations, { assert_eq!(cols.len(), 1); @@ -350,7 +350,7 @@ where // 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 current_block = self.values.last_mut().unwrap(); let block_offset = current_block.len() as u64; current_block.push(Default::default()); @@ -391,7 +391,7 @@ where // 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 current_block = self.values.last_mut().unwrap(); let block_offset = current_block.len() as u64; current_block.push(key); From e3ba95c95363e98aa4d26f0e64ba07657d45b36d Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 6 May 2025 11:05:11 +0800 Subject: [PATCH 47/55] suggest inline. --- .../src/aggregate/groups_accumulator/blocks.rs | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs index 867e0c5cea13..357fb903babf 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -44,6 +44,7 @@ pub struct Blocks { } impl Blocks { + #[inline] pub fn new(block_size: Option) -> Self { Self { inner: Vec::new(), @@ -121,6 +122,7 @@ impl Blocks { } } + #[inline] pub fn pop_block(&mut self) -> Option { if self.next_emit_block_id >= self.inner.len() { return None; @@ -133,18 +135,22 @@ impl Blocks { Some(emit_blk) } + #[inline] pub fn len(&self) -> usize { self.inner.len() } + #[inline] pub fn is_empty(&self) -> bool { self.inner.is_empty() } + #[inline] pub fn iter(&self) -> impl Iterator { self.inner.iter() } + #[inline] pub fn clear(&mut self) { self.inner.clear(); self.next_emit_block_id = 0; @@ -154,12 +160,14 @@ impl Blocks { impl Index for Blocks { type Output = B; + #[inline] fn index(&self, index: usize) -> &Self::Output { &self.inner[index] } } impl IndexMut for Blocks { + #[inline] fn index_mut(&mut self, index: usize) -> &mut Self::Output { &mut self.inner[index] } From 880702618a82373fd84ba06e3f8eae2f3520dcb9 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 6 May 2025 15:24:32 +0800 Subject: [PATCH 48/55] use unsafe to get in Vec. --- .../group_values/single_group_by/primitive.rs | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) 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 b9fe42fac974..bc5f240bd19d 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 @@ -366,8 +366,7 @@ where let block_id = O::get_block_id(*g); let block_offset = O::get_block_offset(*g); self.values - .get(block_id as usize) - .unwrap() + .get_unchecked(block_id as usize) .get_unchecked(block_offset as usize) .is_eq(key) }, @@ -375,8 +374,7 @@ where let block_id = O::get_block_id(*g); let block_offset = O::get_block_offset(*g); self.values - .get(block_id as usize) - .unwrap() + .get_unchecked(block_id as usize) .get_unchecked(block_offset as usize) .hash(state) }, @@ -391,7 +389,10 @@ where // 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.last_mut().unwrap(); + let current_block = unsafe { + let last_index = self.values.len() - 1; + self.values.get_unchecked_mut(last_index) + }; let block_offset = current_block.len() as u64; current_block.push(key); From 62157a9ab896014c17927c7ae97d2177d20e9ec6 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 6 May 2025 17:53:27 +0800 Subject: [PATCH 49/55] optimize rehash. --- .../group_values/single_group_by/primitive.rs | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) 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 bc5f240bd19d..1dba064466b6 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 @@ -90,7 +90,7 @@ pub struct GroupValuesPrimitive { /// /// We don't store the hashes as hashing fixed width primitives /// is fast enough for this not to benefit performance - map: HashTable, + map: HashTable<(u64, u64)>, /// The group index of the null value if any null_group: Option, @@ -220,12 +220,13 @@ where ); let n = n as u64; - self.map.retain(|group_idx| { + self.map.retain(|bucket| { // Decrement group index by n + let group_idx = bucket.0; match group_idx.checked_sub(n) { // Group index was >= n, shift value down Some(sub) => { - *group_idx = sub; + bucket.0 = sub; true } // Group index was < n, so remove from table @@ -363,25 +364,20 @@ where let insert = self.map.entry( hash, |g| unsafe { - let block_id = O::get_block_id(*g); - let block_offset = O::get_block_offset(*g); + let block_id = O::get_block_id(g.0); + let block_offset = O::get_block_offset(g.0); self.values .get_unchecked(block_id as usize) .get_unchecked(block_offset as usize) .is_eq(key) }, - |g| unsafe { - let block_id = O::get_block_id(*g); - let block_offset = O::get_block_offset(*g); - self.values - .get_unchecked(block_id as usize) - .get_unchecked(block_offset as usize) - .hash(state) + |g| { + g.1 }, ); match insert { - hashbrown::hash_table::Entry::Occupied(o) => *o.get(), + hashbrown::hash_table::Entry::Occupied(o) => o.get().0, hashbrown::hash_table::Entry::Vacant(v) => { // Actions before add new group like checking if room is enough before_add_group(&mut self.values); @@ -398,7 +394,7 @@ where // Get group index and finish actions needed it let packed_index = O::pack_index(block_id, block_offset); - v.insert(packed_index); + v.insert((packed_index, hash)); packed_index } } From add409ec15e8adc2ae4fe59f31f3dcf4434dd634 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 7 May 2025 00:14:06 +0800 Subject: [PATCH 50/55] new blocks resize method. --- .../groups_accumulator/accumulate.rs | 38 +-- .../aggregate/groups_accumulator/blocks.rs | 278 ++++++++---------- .../aggregate/groups_accumulator/prim_op.rs | 6 +- 3 files changed, 145 insertions(+), 177 deletions(-) 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 d0304b9c4507..0d189b569467 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -109,11 +109,7 @@ impl NullState { { // ensure the seen_values is big enough (start everything at // "not seen" valid) - let new_block = |block_size: Option| { - BooleanBufferBuilderWrapper(BooleanBufferBuilder::new(block_size.unwrap_or(0))) - }; - self.seen_values.resize(total_num_groups, new_block, false); - + self.seen_values.resize(total_num_groups, false); let seen_values = &mut self.seen_values; accumulate(group_indices, values, opt_filter, |packed_index, value| { let packed_index = packed_index as u64; @@ -149,11 +145,7 @@ impl NullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - let new_block = |block_size: Option| { - BooleanBufferBuilderWrapper(BooleanBufferBuilder::new(block_size.unwrap_or(0))) - }; - self.seen_values.resize(total_num_groups, new_block, false); - + self.seen_values.resize(total_num_groups, false); // These could be made more performant by iterating in chunks of 64 bits at a time let seen_values = &mut self.seen_values; match (values.null_count() > 0, opt_filter) { @@ -332,7 +324,7 @@ impl NullStateAdapter { fn build_cloned_seen_values(&self) -> BooleanBuffer { match self { NullStateAdapter::Flat(null_state) => { - null_state.seen_values[0].finish_cloned() + null_state.seen_values[0].0.finish_cloned() } NullStateAdapter::Blocked(null_state) => { let mut return_builder = BooleanBufferBuilder::new(0); @@ -340,7 +332,7 @@ impl NullStateAdapter { for blk_idx in 0..num_blocks { let builder = &null_state.seen_values[blk_idx]; for idx in 0..builder.len() { - return_builder.append(builder.get_bit(idx)); + return_builder.append(builder.0.get_bit(idx)); } } return_builder.finish() @@ -443,13 +435,23 @@ impl BlockedNullState { impl Block for BooleanBufferBuilderWrapper { 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() } + + fn build(block_size: Option, default_val: bool) -> Self { + if let Some(blk_size) = block_size { + let mut builder = BooleanBufferBuilder::new(blk_size); + builder.append_n(blk_size, default_val); + Self(builder) + } else { + Self(BooleanBufferBuilder::new(0)) + } + } + + fn resize(&mut self, new_len: usize, _default_val: bool) { + self.0.resize(new_len); + } } #[derive(Debug)] @@ -463,7 +465,9 @@ impl Default for BooleanBufferBuilderWrapper { impl Blocks { fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool) { - self[block_id as usize].0.set_bit(block_offset as usize, value); + self[block_id as usize] + .0 + .set_bit(block_offset as usize, value); } fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs index 357fb903babf..ebc94885c7a3 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocks.rs @@ -40,6 +40,7 @@ use datafusion_expr_common::groups_accumulator::EmitTo; pub struct Blocks { inner: Vec, next_emit_block_id: usize, + last_block_len: usize, block_size: Option, } @@ -49,77 +50,30 @@ impl Blocks { Self { inner: Vec::new(), next_emit_block_id: 0, + last_block_len: 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 (mut cur_blk_idx, exist_slots) = if !self.inner.is_empty() { - let cur_blk_idx = self.inner.len() - 1; - let exist_slots = - (self.inner.len() - 1) * block_size + self.inner.last().unwrap().len(); - - (cur_blk_idx, exist_slots) + pub fn resize(&mut self, total_num_groups: usize, default_val: B::T) { + if let Some(block_size) = self.block_size { + let blocks_cap = self.inner.len() * block_size; + if blocks_cap < total_num_groups { + let allocated_blocks = + (total_num_groups - blocks_cap).div_ceil(block_size); + self.inner.extend( + iter::repeat_with(|| B::build(self.block_size, default_val.clone())) + .take(allocated_blocks), + ); + } + self.last_block_len = block_size + total_num_groups - self.inner.len() * block_size; } else { - (0, 0) - }; - - // No new groups, don't need to expand, just return - if exist_slots >= total_num_groups { - return; - } - - // 1. Ensure blks are enough - let exist_blks = self.inner.len(); - let new_blks = total_num_groups.div_ceil(block_size) - exist_blks; - if new_blks > 0 { - for _ in 0..new_blks { - let block = new_block(self.block_size); - self.inner.push(block); + if self.is_empty() { + self.inner.push(B::build(self.block_size, default_val.clone())); } - } - - // 2. Ensure slots are enough - let mut new_slots = total_num_groups - exist_slots; - - // 2.1 Only fill current blk if it may be already enough - let cur_blk_rest_slots = block_size - self.inner[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - self.inner[cur_blk_idx].fill_default_value(new_slots, default_value.clone()); - return; - } - - // 2.2 Fill current blk to full - self.inner[cur_blk_idx] - .fill_default_value(cur_blk_rest_slots, default_value.clone()); - new_slots -= cur_blk_rest_slots; - - // 2.3 Fill complete blks - let complete_blks = new_slots / block_size; - for _ in 0..complete_blks { - 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 = new_slots % block_size; - if rest_slots > 0 { - self.inner - .last_mut() - .unwrap() - .fill_default_value(rest_slots, default_value); - } + let single_block = self.inner.last_mut().unwrap(); + single_block.resize(total_num_groups, default_val.clone()); + }; } #[inline] @@ -129,9 +83,13 @@ impl Blocks { } let emit_block_id = self.next_emit_block_id; - let emit_blk = std::mem::take(&mut self.inner[emit_block_id]); + let mut emit_blk = std::mem::take(&mut self.inner[emit_block_id]); self.next_emit_block_id += 1; + if self.next_emit_block_id == self.inner.len() { + emit_blk.resize(self.last_block_len, B::T::default()); + } + Some(emit_blk) } @@ -180,9 +138,11 @@ impl IndexMut for Blocks { /// to abstract the necessary behaviors of various intermediate result types. /// pub trait Block: Debug + Default { - type T: Clone; + type T: Clone + Default; + + fn build(block_size: Option, default_val: Self::T) -> Self; - fn fill_default_value(&mut self, fill_len: usize, default_value: Self::T); + fn resize(&mut self, new_len: usize, default_val: Self::T); fn len(&self) -> usize; @@ -197,11 +157,19 @@ pub type GeneralBlocks = Blocks>; /// As mentioned in [`GeneralBlocks`], we usually use `Vec` to represent `Block`, /// so we implement `Block` trait for `Vec` -impl Block 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 build(block_size: Option, default_val: Self::T) -> Self { + if let Some(blk_size) = block_size { + vec![default_val; blk_size] + } else { + Vec::new() + } + } + + fn resize(&mut self, new_len: usize, default_val: Self::T) { + self.resize(new_len, default_val); } fn len(&self) -> usize { @@ -209,7 +177,7 @@ impl Block for Vec { } } -impl GeneralBlocks { +impl GeneralBlocks { pub fn emit(&mut self, emit_to: EmitTo) -> Vec { if matches!(emit_to, EmitTo::NextBlock) { assert!( @@ -237,87 +205,87 @@ mod test { 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 0); - - // Test resize after clear in next round - } - } + // #[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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 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.len(), 0); + + // // Test resize after clear in next round + // } + // } } 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 c8c906800a01..644ce3ab35c5 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 @@ -100,12 +100,8 @@ where let values = values[0].as_primitive::(); // Expand to ensure values are large enough - let new_block = |block_size: Option| { - let cap = block_size.unwrap_or(DEFAULT_BLOCK_CAP); - Vec::with_capacity(cap) - }; self.values - .resize(total_num_groups, new_block, self.starting_value); + .resize(total_num_groups, self.starting_value); // NullState dispatches / handles tracking nulls and groups that saw no values self.null_state.accumulate( From c7ce363ba6346f91972ce5772d7bc3c48d3d4fe8 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 7 May 2025 03:18:11 +0800 Subject: [PATCH 51/55] optimize boolean builder blocks. --- .../groups_accumulator/accumulate.rs | 22 ++++++++++++------- .../aggregate/groups_accumulator/prim_op.rs | 2 -- 2 files changed, 14 insertions(+), 10 deletions(-) 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 0d189b569467..ab5045641756 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -23,7 +23,7 @@ use std::fmt::Debug; use std::marker::PhantomData; use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; -use arrow::buffer::{BooleanBuffer, NullBuffer}; +use arrow::buffer::{BooleanBuffer, MutableBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; @@ -440,13 +440,19 @@ impl Block for BooleanBufferBuilderWrapper { } fn build(block_size: Option, default_val: bool) -> Self { - if let Some(blk_size) = block_size { - let mut builder = BooleanBufferBuilder::new(blk_size); - builder.append_n(blk_size, default_val); - Self(builder) - } else { - Self(BooleanBufferBuilder::new(0)) - } + let builder = match (block_size, default_val) { + (Some(blk_size), true) => { + let mut builder = BooleanBufferBuilder::new(blk_size); + builder.append_n(blk_size, true); + builder + } + (Some(blk_size), false) => BooleanBufferBuilder::new_from_buffer( + MutableBuffer::new_null(blk_size), + blk_size, + ), + (None, _) => BooleanBufferBuilder::new(0), + }; + Self(builder) } fn resize(&mut self, new_len: usize, _default_val: bool) { 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 644ce3ab35c5..9f9243cb88fd 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 @@ -94,8 +94,6 @@ where opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { - const DEFAULT_BLOCK_CAP: usize = 128; - assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); From 13296c1a5240c4cf9321d2c61b32880c5a6d3998 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 8 May 2025 21:05:51 +0800 Subject: [PATCH 52/55] revert BooleanBuffer. --- .../groups_accumulator/accumulate.rs | 1868 +++++++++-------- .../group_index_operations.rs | 10 + 2 files changed, 953 insertions(+), 925 deletions(-) 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 ab5045641756..b883a55b052d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,6 +19,7 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator +use std::cmp; use std::fmt::Debug; use std::marker::PhantomData; @@ -67,17 +68,29 @@ pub struct NullState { /// /// If `seen_values[i]` is false, have not seen any values that /// pass the filter yet for group `i` - seen_values: Blocks, + seen_values: BooleanBufferBuilder, + + block_size: Option, + + emit_context: Option, /// phantom data for required type `` _phantom: PhantomData, } +#[derive(Debug)] +struct EmitBlocksContext { + next_emit_block_id: usize, + last_block_len: usize, + num_blocks: usize, + buffer: BooleanBuffer, +} + impl NullState { /// return the size of all buffers allocated by this null state, not including self pub fn size(&self) -> usize { // capacity is in bits, so convert to bytes - self.seen_values.size() / 8 + self.seen_values.capacity() / 8 } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -109,13 +122,15 @@ impl NullState { { // ensure the seen_values is big enough (start everything at // "not seen" valid) - self.seen_values.resize(total_num_groups, false); - let seen_values = &mut self.seen_values; + let seen_values = + initialize_builder(&mut self.seen_values, total_num_groups, false); + let block_size = self.block_size.unwrap_or_default(); 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); - seen_values.set_bit(block_id, block_offset, true); + let flat_index = O::get_flat_index(block_id, block_offset, block_size); + seen_values.set_bit(flat_index, false); value_fn(block_id, block_offset, value); }); } @@ -143,11 +158,12 @@ impl NullState { let data = values.values(); assert_eq!(data.len(), group_indices.len()); + // These could be made more performant by iterating in chunks of 64 bits at a time // ensure the seen_values is big enough (start everything at // "not seen" valid) - self.seen_values.resize(total_num_groups, false); - // These could be made more performant by iterating in chunks of 64 bits at a time - let seen_values = &mut self.seen_values; + let seen_values = + initialize_builder(&mut self.seen_values, total_num_groups, false); + let block_size = self.block_size.unwrap_or_default(); match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { @@ -158,7 +174,9 @@ impl NullState { 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.set_bit(block_id, block_offset, true); + let flat_index = + O::get_flat_index(block_id, block_offset, block_size); + seen_values.set_bit(flat_index, true); value_fn(block_id, block_offset, new_value) }, ) @@ -175,7 +193,9 @@ impl NullState { 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.set_bit(block_id, block_offset, true); + let flat_index = + O::get_flat_index(block_id, block_offset, block_size); + seen_values.set_bit(flat_index, true); value_fn(block_id, block_offset, new_value); } }) @@ -193,7 +213,9 @@ impl NullState { 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.set_bit(block_id, block_offset, true); + let flat_index = + O::get_flat_index(block_id, block_offset, block_size); + seen_values.set_bit(flat_index, true); value_fn(block_id, block_offset, new_value); } }) @@ -211,7 +233,9 @@ impl NullState { 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.set_bit(block_id, block_offset, true); + let flat_index = + O::get_flat_index(block_id, block_offset, block_size); + seen_values.set_bit(flat_index, true); value_fn(block_id, block_offset, new_value); } } @@ -219,15 +243,22 @@ impl NullState { } } } +} - /// 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) -> NullBuffer { - self.seen_values.emit(emit_to) +/// Ensures that `builder` contains a `BooleanBufferBuilder with at +/// least `total_num_groups`. +/// +/// All new entries are initialized to `default_value` +fn initialize_builder( + builder: &mut BooleanBufferBuilder, + total_num_groups: usize, + default_value: bool, +) -> &mut BooleanBufferBuilder { + if builder.len() < total_num_groups { + let new_groups = total_num_groups - builder.len(); + builder.append_n(new_groups, default_value); } + builder } /// Adapter for supporting dynamic dispatching of [`FlatNullState`] and [`BlockedNullState`]. @@ -307,7 +338,7 @@ impl NullStateAdapter { pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { match self { NullStateAdapter::Flat(null_state) => null_state.build(emit_to), - NullStateAdapter::Blocked(null_state) => null_state.build(emit_to), + NullStateAdapter::Blocked(null_state) => null_state.build(), } } @@ -318,46 +349,44 @@ impl NullStateAdapter { } } - /// Clone and build a single [`BooleanBuffer`] from `seen_values`, - /// only used for testing. - #[cfg(test)] - fn build_cloned_seen_values(&self) -> BooleanBuffer { - match self { - NullStateAdapter::Flat(null_state) => { - null_state.seen_values[0].0.finish_cloned() - } - NullStateAdapter::Blocked(null_state) => { - let mut return_builder = BooleanBufferBuilder::new(0); - let num_blocks = null_state.seen_values.len(); - for blk_idx in 0..num_blocks { - let builder = &null_state.seen_values[blk_idx]; - for idx in 0..builder.len() { - return_builder.append(builder.0.get_bit(idx)); - } - } - return_builder.finish() - } - } - } - - #[cfg(test)] - fn build_all_in_once(&mut self) -> NullBuffer { - 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.seen_values.len(); - for _ in 0..num_blocks { - let blocked_nulls = null_state.build(EmitTo::NextBlock); - for bit in blocked_nulls.inner().iter() { - return_builder.append(bit); - } - } - - NullBuffer::new(return_builder.finish()) - } - } - } + // Clone and build a single [`BooleanBuffer`] from `seen_values`, + // only used for testing. + // #[cfg(test)] + // fn build_cloned_seen_values(&self) -> BooleanBuffer { + // match self { + // NullStateAdapter::Flat(null_state) => null_state.seen_values.finish_cloned(), + // NullStateAdapter::Blocked(null_state) => { + // let mut return_builder = BooleanBufferBuilder::new(0); + // let num_blocks = null_state.seen_values.len(); + // for blk_idx in 0..num_blocks { + // let builder = &null_state.seen_values[blk_idx]; + // for idx in 0..builder.len() { + // return_builder.append(builder.0.get_bit(idx)); + // } + // } + // return_builder.finish() + // } + // } + // } + + // #[cfg(test)] + // fn build_all_in_once(&mut self) -> NullBuffer { + // 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.seen_values.len(); + // for _ in 0..num_blocks { + // let blocked_nulls = null_state.build(EmitTo::NextBlock); + // for bit in blocked_nulls.inner().iter() { + // return_builder.append(bit); + // } + // } + + // NullBuffer::new(return_builder.finish()) + // } + // } + // } } /// [`NullState`] for `flat groups input` @@ -394,8 +423,34 @@ impl FlatNullState { impl Default for FlatNullState { fn default() -> Self { Self { - seen_values: Blocks::new(None), - _phantom: PhantomData {}, + seen_values: BooleanBufferBuilder::new(0), + block_size: None, + emit_context: None, + _phantom: PhantomData, + } + } +} + +impl FlatNullState { + pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { + match emit_to { + EmitTo::All => { + NullBuffer::new(self.seen_values.finish()) + } + EmitTo::First(n) => { + // split off the first N values in seen_values + // + // TODO make this more efficient rather than two + // copies and bitwise manipulation + let nulls = self.seen_values.finish(); + let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); + // reset the existing seen buffer + for seen in nulls.iter().skip(n) { + self.seen_values.append(seen); + } + NullBuffer::new(first_n_null) + } + EmitTo::NextBlock => unreachable!(), } } } @@ -426,87 +481,50 @@ pub type BlockedNullState = NullState; impl BlockedNullState { pub fn new(block_size: usize) -> Self { Self { - seen_values: Blocks::new(Some(block_size)), + seen_values: BooleanBufferBuilder::new(0), + block_size: Some(block_size), + emit_context: None, _phantom: PhantomData {}, } } } -impl Block for BooleanBufferBuilderWrapper { - type T = bool; - - fn len(&self) -> usize { - self.0.len() - } - - fn build(block_size: Option, default_val: bool) -> Self { - let builder = match (block_size, default_val) { - (Some(blk_size), true) => { - let mut builder = BooleanBufferBuilder::new(blk_size); - builder.append_n(blk_size, true); - builder - } - (Some(blk_size), false) => BooleanBufferBuilder::new_from_buffer( - MutableBuffer::new_null(blk_size), - blk_size, - ), - (None, _) => BooleanBufferBuilder::new(0), - }; - Self(builder) - } - - fn resize(&mut self, new_len: usize, _default_val: bool) { - self.0.resize(new_len); - } -} - -#[derive(Debug)] -struct BooleanBufferBuilderWrapper(BooleanBufferBuilder); - -impl Default for BooleanBufferBuilderWrapper { - fn default() -> Self { - Self(BooleanBufferBuilder::new(0)) - } -} - -impl Blocks { - fn set_bit(&mut self, block_id: u32, block_offset: u64, value: bool) { - self[block_id as usize] - .0 - .set_bit(block_offset as usize, value); - } +impl BlockedNullState { + pub fn build(&mut self) -> NullBuffer { + let block_size = self.block_size.unwrap(); + + if self.emit_context.is_none() { + let buffer = self.seen_values.finish(); + let num_blocks = buffer.len().div_ceil(block_size); + let mut last_block_len = buffer.len() % block_size; + last_block_len = if last_block_len > 0 { + last_block_len + } else { + usize::MAX + }; + + self.emit_context = Some(EmitBlocksContext { + next_emit_block_id: 0, + last_block_len, + num_blocks, + buffer, + }); + } - fn emit(&mut self, emit_to: EmitTo) -> NullBuffer { - let nulls = match emit_to { - EmitTo::All | EmitTo::First(_) => self[0].0.finish(), - EmitTo::NextBlock => { - let mut block = self - .pop_block() - .expect("should not try to emit empty blocks"); - block.0.finish() - } - }; + let emit_context = self.emit_context.as_mut().unwrap(); + let cur_emit_block_id = emit_context.next_emit_block_id; + emit_context.next_emit_block_id += 1; - 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].0.append(seen); - } - first_n_null + assert!(cur_emit_block_id < emit_context.num_blocks); + let slice_offset = cur_emit_block_id * block_size; + let slice_len = if cur_emit_block_id == emit_context.num_blocks - 1 { + cmp::min(emit_context.last_block_len, block_size) } else { - nulls + block_size }; - NullBuffer::new(nulls) - } - - pub fn size(&self) -> usize { - self.iter().map(|b| b.0.capacity()).sum::() + let emit_block = emit_context.buffer.slice(slice_offset, slice_len); + NullBuffer::new(emit_block) } } @@ -858,795 +876,795 @@ pub fn accumulate_indices( } } -#[cfg(test)] -mod test { - use super::*; - - use arrow::array::{Int32Array, UInt32Array}; - use rand::{rngs::ThreadRng, Rng}; - use std::{cmp, collections::HashSet}; - - #[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(); - - // Test flat style - Fixture { - group_indices, - values, - values_with_nulls, - filter, - block_size: 3, - acc_rounds: 5, - } - .run() - } - - #[test] - fn accumulate_fuzz() { - let mut rng = rand::thread_rng(); - for _ in 0..100 { - Fixture::new_random(&mut rng).run(); - } - } - - /// Values for testing (there are enough values to exercise the 64 bit chunks - struct Fixture { - /// 100..0 - group_indices: Vec, - - /// 10, 20, ... 1010 - values: Vec, - - /// same as values, but every third is null: - /// None, Some(20), Some(30), None ... - values_with_nulls: Vec>, - - /// filter (defaults to None) - filter: BooleanArray, - - /// block size for testing [`BlockedNullState`] - block_size: usize, - - acc_rounds: usize, - } - - impl Fixture { - fn new_random(rng: &mut ThreadRng) -> Self { - // Number of input values in a batch - let num_values: usize = rng.gen_range(1..200); - // number of distinct groups - let num_groups: usize = rng.gen_range(2..1000); - let max_group = num_groups - 1; - - let group_indices: Vec = (0..num_values) - .map(|_| rng.gen_range(0..max_group)) - .collect(); - - let values: Vec = (0..num_values).map(|_| rng.gen()).collect(); - - // random block size - let block_size = rng.gen_range(1..num_groups); - - // random acc rounds - let acc_rounds = rng.gen_range(1..=group_indices.len()); - - // 10% chance of false - // 10% change of null - // 80% chance of true - let filter: BooleanArray = (0..num_values) - .map(|_| { - let filter_value = rng.gen_range(0.0..1.0); - if filter_value < 0.1 { - Some(false) - } else if filter_value < 0.2 { - None - } else { - Some(true) - } - }) - .collect(); - - // random values with random number and location of nulls - // random null percentage - let null_pct: f32 = rng.gen_range(0.0..1.0); - let values_with_nulls: Vec> = (0..num_values) - .map(|_| { - let is_null = null_pct < rng.gen_range(0.0..1.0); - if is_null { - None - } else { - Some(rng.gen()) - } - }) - .collect(); - - Self { - group_indices, - values, - values_with_nulls, - filter, - block_size, - acc_rounds, - } - } - - /// returns `Self::values` an Array - fn values_array(&self) -> UInt32Array { - UInt32Array::from(self.values.clone()) - } - - /// returns `Self::values_with_nulls` as an Array - fn values_with_nulls_array(&self) -> UInt32Array { - UInt32Array::from(self.values_with_nulls.clone()) - } - - /// Calls `NullState::accumulate` and `accumulate_indices` - /// with all combinations of nulls and filter values - fn run(&self) { - let total_num_groups = *self.group_indices.iter().max().unwrap() + 1; - - let group_indices = &self.group_indices; - let values_array = self.values_array(); - let values_with_nulls_array = self.values_with_nulls_array(); - let filter = &self.filter; - - // no null, no filters - Self::accumulate_test( - group_indices, - &values_array, - None, - total_num_groups, - self.block_size, - self.acc_rounds, - ); - - // nulls, no filters - Self::accumulate_test( - group_indices, - &values_with_nulls_array, - None, - total_num_groups, - self.block_size, - self.acc_rounds, - ); - - // no nulls, filters - Self::accumulate_test( - group_indices, - &values_array, - Some(filter), - total_num_groups, - self.block_size, - self.acc_rounds, - ); - - // nulls, filters - Self::accumulate_test( - group_indices, - &values_with_nulls_array, - Some(filter), - total_num_groups, - self.block_size, - self.acc_rounds, - ); - } - - /// Calls `NullState::accumulate` and `accumulate_indices` to - /// ensure it generates the correct values. - /// - fn accumulate_test( - group_indices: &[usize], - 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), - ); - - // 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 - /// accumulate that we compare with the above implementation - fn accumulate_values_test( - group_indices: &[usize], - values: &UInt32Array, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - block_size: Option, - acc_rounds: Option, - ) { - // 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()] - }; - - (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![]; - let mut mock = MockNullState::new(); - - match opt_filter { - None => group_indices.iter().zip(values.iter()).for_each( - |(&group_index, value)| { - if let Some(value) = value { - mock.saw_value(group_index); - expected_values.push((group_index, value)); - } - }, - ), - Some(filter) => { - group_indices - .iter() - .zip(values.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, value), is_included)| { - // if value passed filter - if let Some(true) = is_included { - if let Some(value) = value { - mock.saw_value(group_index); - expected_values.push((group_index, value)); - } - } - }); - } - } - - assert_eq!(accumulated_values, expected_values, - "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - let seen_values = null_state.build_cloned_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_all_in_once(); - - assert_eq!(null_buffer, expected_null_buffer); - } - - // Calls `accumulate_indices` - // and opt_filter and ensures it calls the right values - fn accumulate_indices_test( - group_indices: &[usize], - nulls: Option<&NullBuffer>, - opt_filter: Option<&BooleanArray>, - ) { - let mut accumulated_values = vec![]; - - accumulate_indices(group_indices, nulls, opt_filter, |group_index| { - accumulated_values.push(group_index); - }); - - // Figure out the expected values - let mut expected_values = vec![]; - - match (nulls, opt_filter) { - (None, None) => group_indices.iter().for_each(|&group_index| { - expected_values.push(group_index); - }), - (Some(nulls), None) => group_indices.iter().zip(nulls.iter()).for_each( - |(&group_index, is_valid)| { - if is_valid { - expected_values.push(group_index); - } - }, - ), - (None, Some(filter)) => group_indices.iter().zip(filter.iter()).for_each( - |(&group_index, is_included)| { - if let Some(true) = is_included { - expected_values.push(group_index); - } - }, - ), - (Some(nulls), Some(filter)) => { - group_indices - .iter() - .zip(nulls.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, is_valid), is_included)| { - // if value passed filter - if let (true, Some(true)) = (is_valid, is_included) { - expected_values.push(group_index); - } - }); - } - } - - assert_eq!(accumulated_values, expected_values, - "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - } - - /// This is effectively a different implementation of - /// accumulate_boolean that we compare with the above implementation - fn accumulate_boolean_test( - group_indices: &[usize], - values: &BooleanArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - block_size: Option, - acc_rounds: Option, - ) { - // 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()] - }; - - (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![]; - let mut mock = MockNullState::new(); - - match opt_filter { - None => group_indices.iter().zip(values.iter()).for_each( - |(&group_index, value)| { - if let Some(value) = value { - mock.saw_value(group_index); - expected_values.push((group_index, value)); - } - }, - ), - Some(filter) => { - group_indices - .iter() - .zip(values.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, value), is_included)| { - // if value passed filter - if let Some(true) = is_included { - if let Some(value) = value { - mock.saw_value(group_index); - expected_values.push((group_index, value)); - } - } - }); - } - } - - assert_eq!(accumulated_values, expected_values, - "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); - - let seen_values = null_state.build_cloned_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_all_in_once(); - - assert_eq!(null_buffer, expected_null_buffer); - } - } - - /// Parallel implementation of NullState to check expected values - #[derive(Debug, Default)] - struct MockNullState { - /// group indices that had values that passed the filter - seen_values: HashSet, - } - - impl MockNullState { - fn new() -> Self { - Default::default() - } - - fn saw_value(&mut self, group_index: usize) { - self.seen_values.insert(group_index); - } - - /// did this group index see any input? - fn expected_seen(&self, group_index: usize) -> bool { - self.seen_values.contains(&group_index) - } - - /// Validate that the seen_values matches self.seen_values - fn validate_seen_values(&self, seen_values: &BooleanBuffer) { - for (group_index, is_seen) in seen_values.iter().enumerate() { - let expected_seen = self.expected_seen(group_index); - assert_eq!( - expected_seen, is_seen, - "mismatch at for group {group_index}" - ); - } - } - - /// Create the expected null buffer based on if the input had nulls and a filter - fn expected_null_buffer(&self, total_num_groups: usize) -> NullBuffer { - (0..total_num_groups) - .map(|group_index| self.expected_seen(group_index)) - .collect() - } - } - - #[test] - fn test_accumulate_multiple_no_nulls_no_filter() { - let group_indices = vec![0, 1, 0, 1]; - let values1 = Int32Array::from(vec![1, 2, 3, 4]); - let values2 = Int32Array::from(vec![10, 20, 30, 40]); - let value_columns = [values1, values2]; - - let mut accumulated = vec![]; - accumulate_multiple( - &group_indices, - &value_columns.iter().collect::>(), - None, - |group_idx, batch_idx, columns| { - let values = columns.iter().map(|col| col.value(batch_idx)).collect(); - accumulated.push((group_idx, values)); - }, - ); - - let expected = vec![ - (0, vec![1, 10]), - (1, vec![2, 20]), - (0, vec![3, 30]), - (1, vec![4, 40]), - ]; - assert_eq!(accumulated, expected); - } - - #[test] - fn test_accumulate_multiple_with_nulls() { - let group_indices = vec![0, 1, 0, 1]; - let values1 = Int32Array::from(vec![Some(1), None, Some(3), Some(4)]); - let values2 = Int32Array::from(vec![Some(10), Some(20), None, Some(40)]); - let value_columns = [values1, values2]; - - let mut accumulated = vec![]; - accumulate_multiple( - &group_indices, - &value_columns.iter().collect::>(), - None, - |group_idx, batch_idx, columns| { - let values = columns.iter().map(|col| col.value(batch_idx)).collect(); - accumulated.push((group_idx, values)); - }, - ); - - // Only rows where both columns are non-null should be accumulated - let expected = vec![(0, vec![1, 10]), (1, vec![4, 40])]; - assert_eq!(accumulated, expected); - } - - #[test] - fn test_accumulate_multiple_with_filter() { - let group_indices = vec![0, 1, 0, 1]; - let values1 = Int32Array::from(vec![1, 2, 3, 4]); - let values2 = Int32Array::from(vec![10, 20, 30, 40]); - let value_columns = [values1, values2]; - - let filter = BooleanArray::from(vec![true, false, true, false]); - - let mut accumulated = vec![]; - accumulate_multiple( - &group_indices, - &value_columns.iter().collect::>(), - Some(&filter), - |group_idx, batch_idx, columns| { - let values = columns.iter().map(|col| col.value(batch_idx)).collect(); - accumulated.push((group_idx, values)); - }, - ); - - // Only rows where filter is true should be accumulated - let expected = vec![(0, vec![1, 10]), (0, vec![3, 30])]; - assert_eq!(accumulated, expected); - } - - #[test] - fn test_accumulate_multiple_with_nulls_and_filter() { - let group_indices = vec![0, 1, 0, 1]; - let values1 = Int32Array::from(vec![Some(1), None, Some(3), Some(4)]); - let values2 = Int32Array::from(vec![Some(10), Some(20), None, Some(40)]); - let value_columns = [values1, values2]; - - let filter = BooleanArray::from(vec![true, true, true, false]); - - let mut accumulated = vec![]; - accumulate_multiple( - &group_indices, - &value_columns.iter().collect::>(), - Some(&filter), - |group_idx, batch_idx, columns| { - let values = columns.iter().map(|col| col.value(batch_idx)).collect(); - accumulated.push((group_idx, values)); - }, - ); - - // Only rows where both: - // 1. Filter is true - // 2. Both columns are non-null - // should be accumulated - let expected = [(0, vec![1, 10])]; - assert_eq!(accumulated, expected); - } -} +// #[cfg(test)] +// mod test { +// use super::*; + +// use arrow::array::{Int32Array, UInt32Array}; +// use rand::{rngs::ThreadRng, Rng}; +// use std::{cmp, collections::HashSet}; + +// #[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(); + +// // Test flat style +// Fixture { +// group_indices, +// values, +// values_with_nulls, +// filter, +// block_size: 3, +// acc_rounds: 5, +// } +// .run() +// } + +// #[test] +// fn accumulate_fuzz() { +// let mut rng = rand::thread_rng(); +// for _ in 0..100 { +// Fixture::new_random(&mut rng).run(); +// } +// } + +// /// Values for testing (there are enough values to exercise the 64 bit chunks +// struct Fixture { +// /// 100..0 +// group_indices: Vec, + +// /// 10, 20, ... 1010 +// values: Vec, + +// /// same as values, but every third is null: +// /// None, Some(20), Some(30), None ... +// values_with_nulls: Vec>, + +// /// filter (defaults to None) +// filter: BooleanArray, + +// /// block size for testing [`BlockedNullState`] +// block_size: usize, + +// acc_rounds: usize, +// } + +// impl Fixture { +// fn new_random(rng: &mut ThreadRng) -> Self { +// // Number of input values in a batch +// let num_values: usize = rng.gen_range(1..200); +// // number of distinct groups +// let num_groups: usize = rng.gen_range(2..1000); +// let max_group = num_groups - 1; + +// let group_indices: Vec = (0..num_values) +// .map(|_| rng.gen_range(0..max_group)) +// .collect(); + +// let values: Vec = (0..num_values).map(|_| rng.gen()).collect(); + +// // random block size +// let block_size = rng.gen_range(1..num_groups); + +// // random acc rounds +// let acc_rounds = rng.gen_range(1..=group_indices.len()); + +// // 10% chance of false +// // 10% change of null +// // 80% chance of true +// let filter: BooleanArray = (0..num_values) +// .map(|_| { +// let filter_value = rng.gen_range(0.0..1.0); +// if filter_value < 0.1 { +// Some(false) +// } else if filter_value < 0.2 { +// None +// } else { +// Some(true) +// } +// }) +// .collect(); + +// // random values with random number and location of nulls +// // random null percentage +// let null_pct: f32 = rng.gen_range(0.0..1.0); +// let values_with_nulls: Vec> = (0..num_values) +// .map(|_| { +// let is_null = null_pct < rng.gen_range(0.0..1.0); +// if is_null { +// None +// } else { +// Some(rng.gen()) +// } +// }) +// .collect(); + +// Self { +// group_indices, +// values, +// values_with_nulls, +// filter, +// block_size, +// acc_rounds, +// } +// } + +// /// returns `Self::values` an Array +// fn values_array(&self) -> UInt32Array { +// UInt32Array::from(self.values.clone()) +// } + +// /// returns `Self::values_with_nulls` as an Array +// fn values_with_nulls_array(&self) -> UInt32Array { +// UInt32Array::from(self.values_with_nulls.clone()) +// } + +// /// Calls `NullState::accumulate` and `accumulate_indices` +// /// with all combinations of nulls and filter values +// fn run(&self) { +// let total_num_groups = *self.group_indices.iter().max().unwrap() + 1; + +// let group_indices = &self.group_indices; +// let values_array = self.values_array(); +// let values_with_nulls_array = self.values_with_nulls_array(); +// let filter = &self.filter; + +// // no null, no filters +// Self::accumulate_test( +// group_indices, +// &values_array, +// None, +// total_num_groups, +// self.block_size, +// self.acc_rounds, +// ); + +// // nulls, no filters +// Self::accumulate_test( +// group_indices, +// &values_with_nulls_array, +// None, +// total_num_groups, +// self.block_size, +// self.acc_rounds, +// ); + +// // no nulls, filters +// Self::accumulate_test( +// group_indices, +// &values_array, +// Some(filter), +// total_num_groups, +// self.block_size, +// self.acc_rounds, +// ); + +// // nulls, filters +// Self::accumulate_test( +// group_indices, +// &values_with_nulls_array, +// Some(filter), +// total_num_groups, +// self.block_size, +// self.acc_rounds, +// ); +// } + +// /// Calls `NullState::accumulate` and `accumulate_indices` to +// /// ensure it generates the correct values. +// /// +// fn accumulate_test( +// group_indices: &[usize], +// 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), +// ); + +// // 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 +// /// accumulate that we compare with the above implementation +// fn accumulate_values_test( +// group_indices: &[usize], +// values: &UInt32Array, +// opt_filter: Option<&BooleanArray>, +// total_num_groups: usize, +// block_size: Option, +// acc_rounds: Option, +// ) { +// // 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()] +// }; + +// (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![]; +// let mut mock = MockNullState::new(); + +// match opt_filter { +// None => group_indices.iter().zip(values.iter()).for_each( +// |(&group_index, value)| { +// if let Some(value) = value { +// mock.saw_value(group_index); +// expected_values.push((group_index, value)); +// } +// }, +// ), +// Some(filter) => { +// group_indices +// .iter() +// .zip(values.iter()) +// .zip(filter.iter()) +// .for_each(|((&group_index, value), is_included)| { +// // if value passed filter +// if let Some(true) = is_included { +// if let Some(value) = value { +// mock.saw_value(group_index); +// expected_values.push((group_index, value)); +// } +// } +// }); +// } +// } + +// assert_eq!(accumulated_values, expected_values, +// "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); +// let seen_values = null_state.build_cloned_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_all_in_once(); + +// assert_eq!(null_buffer, expected_null_buffer); +// } + +// // Calls `accumulate_indices` +// // and opt_filter and ensures it calls the right values +// fn accumulate_indices_test( +// group_indices: &[usize], +// nulls: Option<&NullBuffer>, +// opt_filter: Option<&BooleanArray>, +// ) { +// let mut accumulated_values = vec![]; + +// accumulate_indices(group_indices, nulls, opt_filter, |group_index| { +// accumulated_values.push(group_index); +// }); + +// // Figure out the expected values +// let mut expected_values = vec![]; + +// match (nulls, opt_filter) { +// (None, None) => group_indices.iter().for_each(|&group_index| { +// expected_values.push(group_index); +// }), +// (Some(nulls), None) => group_indices.iter().zip(nulls.iter()).for_each( +// |(&group_index, is_valid)| { +// if is_valid { +// expected_values.push(group_index); +// } +// }, +// ), +// (None, Some(filter)) => group_indices.iter().zip(filter.iter()).for_each( +// |(&group_index, is_included)| { +// if let Some(true) = is_included { +// expected_values.push(group_index); +// } +// }, +// ), +// (Some(nulls), Some(filter)) => { +// group_indices +// .iter() +// .zip(nulls.iter()) +// .zip(filter.iter()) +// .for_each(|((&group_index, is_valid), is_included)| { +// // if value passed filter +// if let (true, Some(true)) = (is_valid, is_included) { +// expected_values.push(group_index); +// } +// }); +// } +// } + +// assert_eq!(accumulated_values, expected_values, +// "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); +// } + +// /// This is effectively a different implementation of +// /// accumulate_boolean that we compare with the above implementation +// fn accumulate_boolean_test( +// group_indices: &[usize], +// values: &BooleanArray, +// opt_filter: Option<&BooleanArray>, +// total_num_groups: usize, +// block_size: Option, +// acc_rounds: Option, +// ) { +// // 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()] +// }; + +// (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![]; +// let mut mock = MockNullState::new(); + +// match opt_filter { +// None => group_indices.iter().zip(values.iter()).for_each( +// |(&group_index, value)| { +// if let Some(value) = value { +// mock.saw_value(group_index); +// expected_values.push((group_index, value)); +// } +// }, +// ), +// Some(filter) => { +// group_indices +// .iter() +// .zip(values.iter()) +// .zip(filter.iter()) +// .for_each(|((&group_index, value), is_included)| { +// // if value passed filter +// if let Some(true) = is_included { +// if let Some(value) = value { +// mock.saw_value(group_index); +// expected_values.push((group_index, value)); +// } +// } +// }); +// } +// } + +// assert_eq!(accumulated_values, expected_values, +// "\n\naccumulated_values:{accumulated_values:#?}\n\nexpected_values:{expected_values:#?}"); + +// let seen_values = null_state.build_cloned_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_all_in_once(); + +// assert_eq!(null_buffer, expected_null_buffer); +// } +// } + +// /// Parallel implementation of NullState to check expected values +// #[derive(Debug, Default)] +// struct MockNullState { +// /// group indices that had values that passed the filter +// seen_values: HashSet, +// } + +// impl MockNullState { +// fn new() -> Self { +// Default::default() +// } + +// fn saw_value(&mut self, group_index: usize) { +// self.seen_values.insert(group_index); +// } + +// /// did this group index see any input? +// fn expected_seen(&self, group_index: usize) -> bool { +// self.seen_values.contains(&group_index) +// } + +// /// Validate that the seen_values matches self.seen_values +// fn validate_seen_values(&self, seen_values: &BooleanBuffer) { +// for (group_index, is_seen) in seen_values.iter().enumerate() { +// let expected_seen = self.expected_seen(group_index); +// assert_eq!( +// expected_seen, is_seen, +// "mismatch at for group {group_index}" +// ); +// } +// } + +// /// Create the expected null buffer based on if the input had nulls and a filter +// fn expected_null_buffer(&self, total_num_groups: usize) -> NullBuffer { +// (0..total_num_groups) +// .map(|group_index| self.expected_seen(group_index)) +// .collect() +// } +// } + +// #[test] +// fn test_accumulate_multiple_no_nulls_no_filter() { +// let group_indices = vec![0, 1, 0, 1]; +// let values1 = Int32Array::from(vec![1, 2, 3, 4]); +// let values2 = Int32Array::from(vec![10, 20, 30, 40]); +// let value_columns = [values1, values2]; + +// let mut accumulated = vec![]; +// accumulate_multiple( +// &group_indices, +// &value_columns.iter().collect::>(), +// None, +// |group_idx, batch_idx, columns| { +// let values = columns.iter().map(|col| col.value(batch_idx)).collect(); +// accumulated.push((group_idx, values)); +// }, +// ); + +// let expected = vec![ +// (0, vec![1, 10]), +// (1, vec![2, 20]), +// (0, vec![3, 30]), +// (1, vec![4, 40]), +// ]; +// assert_eq!(accumulated, expected); +// } + +// #[test] +// fn test_accumulate_multiple_with_nulls() { +// let group_indices = vec![0, 1, 0, 1]; +// let values1 = Int32Array::from(vec![Some(1), None, Some(3), Some(4)]); +// let values2 = Int32Array::from(vec![Some(10), Some(20), None, Some(40)]); +// let value_columns = [values1, values2]; + +// let mut accumulated = vec![]; +// accumulate_multiple( +// &group_indices, +// &value_columns.iter().collect::>(), +// None, +// |group_idx, batch_idx, columns| { +// let values = columns.iter().map(|col| col.value(batch_idx)).collect(); +// accumulated.push((group_idx, values)); +// }, +// ); + +// // Only rows where both columns are non-null should be accumulated +// let expected = vec![(0, vec![1, 10]), (1, vec![4, 40])]; +// assert_eq!(accumulated, expected); +// } + +// #[test] +// fn test_accumulate_multiple_with_filter() { +// let group_indices = vec![0, 1, 0, 1]; +// let values1 = Int32Array::from(vec![1, 2, 3, 4]); +// let values2 = Int32Array::from(vec![10, 20, 30, 40]); +// let value_columns = [values1, values2]; + +// let filter = BooleanArray::from(vec![true, false, true, false]); + +// let mut accumulated = vec![]; +// accumulate_multiple( +// &group_indices, +// &value_columns.iter().collect::>(), +// Some(&filter), +// |group_idx, batch_idx, columns| { +// let values = columns.iter().map(|col| col.value(batch_idx)).collect(); +// accumulated.push((group_idx, values)); +// }, +// ); + +// // Only rows where filter is true should be accumulated +// let expected = vec![(0, vec![1, 10]), (0, vec![3, 30])]; +// assert_eq!(accumulated, expected); +// } + +// #[test] +// fn test_accumulate_multiple_with_nulls_and_filter() { +// let group_indices = vec![0, 1, 0, 1]; +// let values1 = Int32Array::from(vec![Some(1), None, Some(3), Some(4)]); +// let values2 = Int32Array::from(vec![Some(10), Some(20), None, Some(40)]); +// let value_columns = [values1, values2]; + +// let filter = BooleanArray::from(vec![true, true, true, false]); + +// let mut accumulated = vec![]; +// accumulate_multiple( +// &group_indices, +// &value_columns.iter().collect::>(), +// Some(&filter), +// |group_idx, batch_idx, columns| { +// let values = columns.iter().map(|col| col.value(batch_idx)).collect(); +// accumulated.push((group_idx, values)); +// }, +// ); + +// // Only rows where both: +// // 1. Filter is true +// // 2. Both columns are non-null +// // should be accumulated +// let expected = [(0, vec![1, 10])]; +// assert_eq!(accumulated, expected); +// } +// } 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 index a62b21fa1be1..fbf24bb7bb4e 100644 --- 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 @@ -54,6 +54,8 @@ pub trait GroupIndexOperations: Debug { fn get_block_id(packed_index: u64) -> u32; fn get_block_offset(packed_index: u64) -> u64; + + fn get_flat_index(block_id: u32, block_offset: u64, block_size: usize) -> usize; } #[derive(Debug)] @@ -71,6 +73,10 @@ impl GroupIndexOperations for BlockedGroupIndexOperations { fn get_block_offset(packed_index: u64) -> u64 { (packed_index as u32) as u64 } + + fn get_flat_index(block_id: u32, block_offset: u64, block_size: usize) -> usize { + block_id as usize * block_size + block_offset as usize + } } #[derive(Debug)] @@ -88,4 +94,8 @@ impl GroupIndexOperations for FlatGroupIndexOperations { fn get_block_offset(packed_index: u64) -> u64 { packed_index } + + fn get_flat_index(_block_id: u32, block_offset: u64, _block_size: usize) -> usize { + block_offset as usize + } } From da4c59074f75bd300ccddea2813a5924c868b081 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 8 May 2025 23:41:19 +0800 Subject: [PATCH 53/55] refactor group index computation. --- datafusion-examples/examples/advanced_udaf.rs | 1 - .../groups_accumulator/accumulate.rs | 66 +++++-------- .../aggregate/groups_accumulator/bool_op.rs | 1 - .../group_index_operations.rs | 38 ++------ datafusion/functions-aggregate/src/average.rs | 1 - .../group_values/single_group_by/primitive.rs | 92 +++++++++++-------- 6 files changed, 88 insertions(+), 111 deletions(-) diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 875841a01a62..b3bcb665a7b5 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -258,7 +258,6 @@ impl GroupsAccumulator for GeometricMeanGroupsAccumulator { opt_filter, total_num_groups, |_, group_index, new_value| { - let group_index = group_index as usize; let prod = &mut self.prods[group_index]; *prod = prod.mul_wrapping(new_value); 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 b883a55b052d..3320685ef268 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -118,19 +118,17 @@ impl NullState { mut value_fn: F, ) where T: ArrowPrimitiveType + Send, - F: FnMut(u32, u64, T::Native) + Send, + F: FnMut(usize, usize, T::Native) + Send, { // ensure the seen_values is big enough (start everything at // "not seen" valid) let seen_values = initialize_builder(&mut self.seen_values, total_num_groups, false); let block_size = self.block_size.unwrap_or_default(); - 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); - let flat_index = O::get_flat_index(block_id, block_offset, block_size); - seen_values.set_bit(flat_index, false); + accumulate(group_indices, values, opt_filter, |group_index, value| { + let block_id = O::get_block_id(group_index, block_size); + let block_offset = O::get_block_offset(group_index, block_size); + seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, value); }); } @@ -153,7 +151,7 @@ impl NullState { total_num_groups: usize, mut value_fn: F, ) where - F: FnMut(u32, u64, bool) + Send, + F: FnMut(usize, usize, bool) + Send, { let data = values.values(); assert_eq!(data.len(), group_indices.len()); @@ -170,13 +168,10 @@ 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( - |(&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); - let flat_index = - O::get_flat_index(block_id, block_offset, block_size); - seen_values.set_bit(flat_index, true); + |(&group_index, new_value)| { + let block_id = O::get_block_id(group_index, block_size); + let block_offset = O::get_block_offset(group_index, block_size); + seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value) }, ) @@ -188,14 +183,11 @@ impl NullState { .iter() .zip(data.iter()) .zip(nulls.iter()) - .for_each(|((&packed_index, new_value), is_valid)| { + .for_each(|((&group_index, new_value), is_valid)| { if is_valid { - 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); - let flat_index = - O::get_flat_index(block_id, block_offset, block_size); - seen_values.set_bit(flat_index, true); + let block_id = O::get_block_id(group_index, block_size); + let block_offset = O::get_block_offset(group_index, block_size); + seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value); } }) @@ -208,14 +200,11 @@ impl NullState { .iter() .zip(data.iter()) .zip(filter.iter()) - .for_each(|((&packed_index, new_value), filter_value)| { + .for_each(|((&group_index, new_value), filter_value)| { if let Some(true) = filter_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); - let flat_index = - O::get_flat_index(block_id, block_offset, block_size); - seen_values.set_bit(flat_index, true); + let block_id = O::get_block_id(group_index, block_size); + let block_offset = O::get_block_offset(group_index, block_size); + seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value); } }) @@ -227,15 +216,12 @@ impl NullState { .iter() .zip(group_indices.iter()) .zip(values.iter()) - .for_each(|((filter_value, &packed_index), new_value)| { + .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = 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); - let flat_index = - O::get_flat_index(block_id, block_offset, block_size); - seen_values.set_bit(flat_index, true); + let block_id = O::get_block_id(group_index, block_size); + let block_offset = O::get_block_offset(group_index, block_size); + seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value); } } @@ -287,7 +273,7 @@ impl NullStateAdapter { value_fn: F, ) where T: ArrowPrimitiveType + Send, - F: FnMut(u32, u64, T::Native) + Send, + F: FnMut(usize, usize, T::Native) + Send, { match self { NullStateAdapter::Flat(null_state) => null_state.accumulate( @@ -315,7 +301,7 @@ impl NullStateAdapter { total_num_groups: usize, value_fn: F, ) where - F: FnMut(u32, u64, bool) + Send, + F: FnMut(usize, usize, bool) + Send, { match self { NullStateAdapter::Flat(null_state) => null_state.accumulate_boolean( @@ -434,9 +420,7 @@ impl Default for FlatNullState { impl FlatNullState { pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { match emit_to { - EmitTo::All => { - NullBuffer::new(self.seen_values.finish()) - } + EmitTo::All => NullBuffer::new(self.seen_values.finish()), EmitTo::First(n) => { // split off the first N values in seen_values // 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 c3713489603e..395fe6df9fbb 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 @@ -95,7 +95,6 @@ where opt_filter, total_num_groups, |_, 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); 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 index fbf24bb7bb4e..05c47a8875bb 100644 --- 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 @@ -49,33 +49,21 @@ use std::fmt::Debug; /// 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(group_index: usize, block_size: usize) -> usize; - fn get_block_id(packed_index: u64) -> u32; - - fn get_block_offset(packed_index: u64) -> u64; - - fn get_flat_index(block_id: u32, block_offset: u64, block_size: usize) -> usize; + fn get_block_offset(group_index: usize, block_size: usize) -> usize; } #[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 + fn get_block_id(group_index: usize, block_size: usize) -> usize { + group_index / block_size } - fn get_flat_index(block_id: u32, block_offset: u64, block_size: usize) -> usize { - block_id as usize * block_size + block_offset as usize + fn get_block_offset(group_index: usize, block_size: usize) -> usize { + group_index % block_size } } @@ -83,19 +71,11 @@ impl GroupIndexOperations for BlockedGroupIndexOperations { 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 { + fn get_block_id(_group_index: usize, _block_size: usize) -> usize { 0 } - fn get_block_offset(packed_index: u64) -> u64 { - packed_index - } - - fn get_flat_index(_block_id: u32, block_offset: u64, _block_size: usize) -> usize { - block_offset as usize + fn get_block_offset(group_index: usize, _block_size: usize) -> usize { + group_index } } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 809a58097505..a7a22a07cf08 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -589,7 +589,6 @@ where opt_filter, total_num_groups, |_, group_index, new_value| { - let group_index = group_index as usize; let sum = &mut self.sums[group_index]; *sum = sum.add_wrapping(new_value); 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 d1f99e57ca9f..efcb171b4d48 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 @@ -93,10 +93,10 @@ pub struct GroupValuesPrimitive { /// More details can see: /// /// - map: HashTable<(u64, u64)>, + map: HashTable<(usize, u64)>, /// The group index of the null value if any - null_group: Option, + null_group: Option, /// The values for each group index values: Vec>, @@ -115,6 +115,8 @@ pub struct GroupValuesPrimitive { /// `Vec` if of `blk_size` len, and we call it a `block` /// block_size: Option, + + num_groups: usize, } impl GroupValuesPrimitive { @@ -134,6 +136,7 @@ impl GroupValuesPrimitive { null_group: None, random_state: Default::default(), block_size: None, + num_groups: 0, } } } @@ -180,7 +183,8 @@ where } fn len(&self) -> usize { - self.values.iter().map(|block| block.len()).sum::() + self.num_groups + // self.values.iter().map(|block| block.len()).sum::() } fn emit(&mut self, emit_to: EmitTo) -> Result> { @@ -222,7 +226,6 @@ where "only support EmitTo::First in flat mode" ); - let n = n as u64; self.map.retain(|bucket| { // Decrement group index by n let group_idx = bucket.0; @@ -256,10 +259,9 @@ where // Emitting in blocked mode // =============================================== EmitTo::NextBlock => { - assert!( - self.block_size.is_some(), - "only support EmitTo::Next in blocked group values" - ); + let block_size = self + .block_size + .expect("only support EmitTo::Next in blocked group values"); // Similar as `EmitTo:All`, we will clear the old index infos both // in `map` and `null_group` @@ -271,10 +273,16 @@ where self.next_emit_block_id += 1; // Check if `null` is in current block - let null_block_pair_opt = self.null_group.map(|packed_idx| { + let null_block_pair_opt = self.null_group.map(|group_index| { ( - BlockedGroupIndexOperations::get_block_id(packed_idx), - BlockedGroupIndexOperations::get_block_offset(packed_idx), + BlockedGroupIndexOperations::get_block_id( + group_index, + block_size, + ), + BlockedGroupIndexOperations::get_block_offset( + group_index, + block_size, + ), ) }); let null_idx = match null_block_pair_opt { @@ -288,6 +296,7 @@ where } }; + self.num_groups -= array.len(); Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) } @@ -301,10 +310,15 @@ where let single_block = self.values.last_mut().unwrap(); single_block.clear(); single_block.shrink_to(count); + } else { + self.values.clear(); } self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared + self.null_group = None; + self.next_emit_block_id = 0; + self.num_groups = 0; } fn supports_blocked_groups(&self) -> bool { @@ -317,6 +331,7 @@ where self.null_group = None; self.block_size = block_size; self.next_emit_block_id = 0; + self.num_groups = 0; // As mentioned above, we ensure the `single block` always exist // in `flat mode` @@ -345,6 +360,7 @@ where assert_eq!(cols.len(), 1); groups.clear(); + let block_size = self.block_size.unwrap_or_default(); for v in cols[0].as_primitive::() { let group_index = match v { None => *self.null_group.get_or_insert_with(|| { @@ -353,13 +369,15 @@ where // 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.last_mut().unwrap(); - let block_offset = current_block.len() as u64; current_block.push(Default::default()); + // Compute group index + let group_index = self.num_groups; + self.num_groups += 1; + // Get group index and finish actions needed it - O::pack_index(block_id, block_offset) + group_index }), Some(key) => { let state = &self.random_state; @@ -367,16 +385,14 @@ where let insert = self.map.entry( hash, |g| unsafe { - let block_id = O::get_block_id(g.0); - let block_offset = O::get_block_offset(g.0); + let block_id = O::get_block_id(g.0, block_size); + let block_offset = O::get_block_offset(g.0, block_size); self.values - .get_unchecked(block_id as usize) - .get_unchecked(block_offset as usize) + .get_unchecked(block_id) + .get_unchecked(block_offset) .is_eq(key) }, - |g| { - g.1 - }, + |g| g.1, ); match insert { @@ -387,23 +403,21 @@ where // 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 = unsafe { - let last_index = self.values.len() - 1; - self.values.get_unchecked_mut(last_index) - }; - let block_offset = current_block.len() as u64; + let current_block = self.values.last_mut().unwrap(); 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 + // Compute group index + let group_index = self.num_groups; + self.num_groups += 1; + + v.insert((group_index, hash)); + group_index } } } }; - groups.push(group_index as usize) + + groups.push(group_index) } Ok(()) } @@ -532,10 +546,11 @@ mod tests { .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); + for (&group_index, value) in group_indices.iter().zip(data1.iter()) { + let block_id = + BlockedGroupIndexOperations::get_block_id(group_index, block_size); let block_offset = - BlockedGroupIndexOperations::get_block_offset(packed_index as u64); + BlockedGroupIndexOperations::get_block_offset(group_index, block_size); let flatten_index = block_id as usize * block_size + block_offset as usize; expected.insert(flatten_index, value); } @@ -557,10 +572,11 @@ mod tests { .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); + for (&group_index, value) in group_indices.iter().zip(data2.iter()) { + let block_id = + BlockedGroupIndexOperations::get_block_id(group_index, block_size); let block_offset = - BlockedGroupIndexOperations::get_block_offset(packed_index as u64); + BlockedGroupIndexOperations::get_block_offset(group_index, block_size); let flatten_index = block_id as usize * block_size + block_offset as usize; expected.insert(flatten_index, value); } From 9d0b73bb53ed81d9c9a93904863ab02379a65864 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 9 May 2025 02:15:57 +0800 Subject: [PATCH 54/55] extract `EmitBlocksContext` to common, and prevent all new updates during emitting. --- .../expr-common/src/groups_accumulator.rs | 58 ++++++++++++++++++ .../group_values/single_group_by/primitive.rs | 61 +++++++++++++++---- 2 files changed, 107 insertions(+), 12 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 53bcd094a205..2341f3909251 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -64,6 +64,64 @@ impl EmitTo { } } +/// Emitting context used in blocked management +#[derive(Debug, Default)] +pub struct EmitBlocksContext { + /// Mark if it is during blocks emitting, if so states can't + /// be updated until all blocks are emitted + pub emitting: bool, + + /// Idx of next emitted block + pub next_emit_block: usize, + + /// Number of blocks needed to emit + pub num_blocks: usize, +} + +impl EmitBlocksContext { + #[inline] + pub fn new() -> Self { + Self::default() + } + + #[inline] + pub fn start_emit(&mut self, num_blocks: usize) { + self.emitting = true; + self.num_blocks = num_blocks; + } + + #[inline] + pub fn emitting(&self) -> bool { + self.emitting + } + + #[inline] + pub fn all_emitted(&self) -> bool { + self.next_emit_block == self.num_blocks + } + + #[inline] + pub fn cur_emit_block(&self) -> usize { + assert!(self.emitting, "must start emit first"); + self.next_emit_block + } + + #[inline] + pub fn advance_emit_block(&mut self) { + assert!(self.emitting, "must start emit first"); + if self.next_emit_block < self.num_blocks { + self.next_emit_block += 1; + } + } + + #[inline] + pub fn reset(&mut self) { + self.emitting = false; + self.next_emit_block = 0; + self.num_blocks = 0; + } +} + /// `GroupsAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// 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 efcb171b4d48..29bf69d644d9 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 @@ -26,6 +26,7 @@ use arrow::datatypes::{i256, DataType}; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::proxy::VecAllocExt; +use datafusion_expr::groups_accumulator::EmitBlocksContext; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::group_index_operations::{ BlockedGroupIndexOperations, FlatGroupIndexOperations, GroupIndexOperations, @@ -101,8 +102,6 @@ pub struct GroupValuesPrimitive { /// The values for each group index values: Vec>, - next_emit_block_id: usize, - /// The random state used to generate hashes random_state: RandomState, @@ -116,7 +115,14 @@ pub struct GroupValuesPrimitive { /// block_size: Option, + /// Number of current storing groups + /// + /// We maintain it to avoid the expansive dynamic computation of + /// `groups number` and `target group index` in `blocked approach` num_groups: usize, + + /// Context used in emitting in `blocked approach` + emit_blocks_ctx: EmitBlocksContext, } impl GroupValuesPrimitive { @@ -132,11 +138,11 @@ impl GroupValuesPrimitive { data_type, map: HashTable::with_capacity(128), values, - next_emit_block_id: 0, null_group: None, random_state: Default::default(), block_size: None, num_groups: 0, + emit_blocks_ctx: EmitBlocksContext::new(), } } } @@ -146,6 +152,11 @@ where T::Native: HashValue, { fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()> { + assert!( + !self.emit_blocks_ctx.emitting(), + "can not update groups during emitting" + ); + if let Some(block_size) = self.block_size { let before_add_group = |group_values: &mut Vec>| { if group_values.is_empty() @@ -184,7 +195,6 @@ where fn len(&self) -> usize { self.num_groups - // self.values.iter().map(|block| block.len()).sum::() } fn emit(&mut self, emit_to: EmitTo) -> Result> { @@ -263,14 +273,27 @@ where .block_size .expect("only support EmitTo::Next in blocked group values"); + // To mark the emitting has already started, and prevent new updates + if !self.emit_blocks_ctx.emitting() { + let num_blocks = self.values.len(); + self.emit_blocks_ctx.start_emit(num_blocks); + } + // Similar as `EmitTo:All`, we will clear the old index infos both // in `map` and `null_group` self.map.clear(); - // Get current emit block id firstly - let emit_block_id = self.next_emit_block_id; + // Get current emit block idx firstly + let emit_block_id = self.emit_blocks_ctx.cur_emit_block(); let emit_blk = std::mem::take(&mut self.values[emit_block_id]); - self.next_emit_block_id += 1; + // And then we advance the block idx + self.emit_blocks_ctx.advance_emit_block(); + // Finally we check if all blocks emitted, if so, we reset the + // emit context to allow new updates + if self.emit_blocks_ctx.all_emitted() { + self.emit_blocks_ctx.reset(); + self.values.clear(); + } // Check if `null` is in current block let null_block_pair_opt = self.null_group.map(|group_index| { @@ -287,6 +310,8 @@ where }); let null_idx = match null_block_pair_opt { Some((blk_id, blk_offset)) if blk_id as usize == emit_block_id => { + // Clear `null_group` mapping + self.null_group = None; Some(blk_offset as usize) } _ => None, @@ -296,13 +321,16 @@ where } }; + // Maintain number of groups self.num_groups -= array.len(); + Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) } fn clear_shrink(&mut self, batch: &RecordBatch) { let count = batch.num_rows(); + // Clear values // TODO: Only reserve room of values in `flat mode` currently, // we may need to consider it again when supporting spilling // for `blocked mode`. @@ -314,10 +342,13 @@ where self.values.clear(); } + // Clear mappings self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared self.null_group = None; - self.next_emit_block_id = 0; + + // Clear helping structures + self.emit_blocks_ctx.reset(); self.num_groups = 0; } @@ -326,11 +357,15 @@ where } fn alter_block_size(&mut self, block_size: Option) -> Result<()> { - self.map.clear(); + // Clear values self.values.clear(); + + // Clear mappings + self.map.clear(); self.null_group = None; - self.block_size = block_size; - self.next_emit_block_id = 0; + + // Clear helping structures + self.emit_blocks_ctx.reset(); self.num_groups = 0; // As mentioned above, we ensure the `single block` always exist @@ -338,6 +373,7 @@ where if block_size.is_none() { self.values.push(Vec::new()); } + self.block_size = block_size; Ok(()) } @@ -430,8 +466,9 @@ mod tests { use crate::aggregates::group_values::single_group_by::primitive::GroupValuesPrimitive; use crate::aggregates::group_values::GroupValues; - use arrow::array::{AsArray, UInt32Array}; + use arrow::array::{AsArray, RecordBatch, UInt32Array}; use arrow::datatypes::{DataType, UInt32Type}; + use arrow_schema::Schema; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::group_index_operations::{ BlockedGroupIndexOperations, GroupIndexOperations, From 7f529b97abffcc87a4aa293a864bdc9be7cac426 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 9 May 2025 03:49:24 +0800 Subject: [PATCH 55/55] use `EmitBlocksContext` to refactor `BlockedNullState`. --- .../groups_accumulator/accumulate.rs | 153 +++++++++++++----- 1 file changed, 115 insertions(+), 38 deletions(-) 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 3320685ef268..92423702b3a0 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -27,7 +27,7 @@ use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, MutableBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_expr_common::groups_accumulator::EmitTo; +use datafusion_expr_common::groups_accumulator::{EmitBlocksContext, EmitTo}; use crate::aggregate::groups_accumulator::blocks::{Block, Blocks}; use crate::aggregate::groups_accumulator::group_index_operations::{ @@ -72,20 +72,10 @@ pub struct NullState { block_size: Option, - emit_context: Option, - /// phantom data for required type `` _phantom: PhantomData, } -#[derive(Debug)] -struct EmitBlocksContext { - next_emit_block_id: usize, - last_block_len: usize, - num_blocks: usize, - buffer: BooleanBuffer, -} - impl NullState { /// return the size of all buffers allocated by this null state, not including self pub fn size(&self) -> usize { @@ -186,7 +176,8 @@ impl NullState { .for_each(|((&group_index, new_value), is_valid)| { if is_valid { let block_id = O::get_block_id(group_index, block_size); - let block_offset = O::get_block_offset(group_index, block_size); + let block_offset = + O::get_block_offset(group_index, block_size); seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value); } @@ -203,7 +194,8 @@ impl NullState { .for_each(|((&group_index, new_value), filter_value)| { if let Some(true) = filter_value { let block_id = O::get_block_id(group_index, block_size); - let block_offset = O::get_block_offset(group_index, block_size); + let block_offset = + O::get_block_offset(group_index, block_size); seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value); } @@ -220,7 +212,8 @@ impl NullState { if let Some(true) = filter_value { if let Some(new_value) = new_value { let block_id = O::get_block_id(group_index, block_size); - let block_offset = O::get_block_offset(group_index, block_size); + let block_offset = + O::get_block_offset(group_index, block_size); seen_values.set_bit(group_index, false); value_fn(block_id, block_offset, new_value); } @@ -264,6 +257,7 @@ impl NullStateAdapter { } } + #[inline] pub fn accumulate( &mut self, group_indices: &[usize], @@ -293,6 +287,7 @@ impl NullStateAdapter { } } + #[inline] pub fn accumulate_boolean( &mut self, group_indices: &[usize], @@ -321,6 +316,7 @@ impl NullStateAdapter { } } + #[inline] pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { match self { NullStateAdapter::Flat(null_state) => null_state.build(emit_to), @@ -328,6 +324,7 @@ impl NullStateAdapter { } } + #[inline] pub fn size(&self) -> usize { match self { NullStateAdapter::Flat(null_state) => null_state.size(), @@ -411,7 +408,6 @@ impl Default for FlatNullState { Self { seen_values: BooleanBufferBuilder::new(0), block_size: None, - emit_context: None, _phantom: PhantomData, } } @@ -460,25 +456,87 @@ impl FlatNullState { /// /// [`GroupsAccumulator::supports_blocked_groups`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator::supports_blocked_groups /// -pub type BlockedNullState = NullState; +#[derive(Debug)] +pub struct BlockedNullState { + inner: NullState, + emit_ctx: NullsEmitContext, +} + +#[derive(Debug, Default)] +struct NullsEmitContext { + base_ctx: EmitBlocksContext, + last_block_len: usize, + buffer: Option, +} + +impl NullsEmitContext { + fn new() -> Self { + Self::default() + } +} impl BlockedNullState { pub fn new(block_size: usize) -> Self { - Self { + let inner = NullState { seen_values: BooleanBufferBuilder::new(0), block_size: Some(block_size), - emit_context: None, _phantom: PhantomData {}, - } + }; + + let emit_ctx = NullsEmitContext::new(); + + Self { inner, emit_ctx } + } + + #[inline] + 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(usize, usize, T::Native) + Send, + { + assert!(!self.emit_ctx.base_ctx.emitting()); + self.inner.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ); + } + + #[inline] + 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(usize, usize, bool) + Send, + { + assert!(!self.emit_ctx.base_ctx.emitting()); + self.inner.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ); } -} -impl BlockedNullState { pub fn build(&mut self) -> NullBuffer { - let block_size = self.block_size.unwrap(); + let block_size = self.inner.block_size.unwrap(); - if self.emit_context.is_none() { - let buffer = self.seen_values.finish(); + if !self.emit_ctx.base_ctx.emitting() { + // Init needed contexts + let buffer = self.inner.seen_values.finish(); let num_blocks = buffer.len().div_ceil(block_size); let mut last_block_len = buffer.len() % block_size; last_block_len = if last_block_len > 0 { @@ -486,30 +544,49 @@ impl BlockedNullState { } else { usize::MAX }; + self.emit_ctx.buffer = Some(buffer); + self.emit_ctx.last_block_len = last_block_len; - self.emit_context = Some(EmitBlocksContext { - next_emit_block_id: 0, - last_block_len, - num_blocks, - buffer, - }); + // Start emit + self.emit_ctx.base_ctx.start_emit(num_blocks); } - let emit_context = self.emit_context.as_mut().unwrap(); - let cur_emit_block_id = emit_context.next_emit_block_id; - emit_context.next_emit_block_id += 1; + // Get current emit block idx + let emit_block_id = self.emit_ctx.base_ctx.cur_emit_block(); + // And then we advance the block idx + self.emit_ctx.base_ctx.advance_emit_block(); - assert!(cur_emit_block_id < emit_context.num_blocks); - let slice_offset = cur_emit_block_id * block_size; - let slice_len = if cur_emit_block_id == emit_context.num_blocks - 1 { - cmp::min(emit_context.last_block_len, block_size) + // Process and generate the emit block + let buffer = self.emit_ctx.buffer.as_ref().unwrap(); + let slice_offset = emit_block_id * block_size; + let slice_len = if self.emit_ctx.base_ctx.all_emitted() { + cmp::min(self.emit_ctx.last_block_len, block_size) } else { block_size }; + let emit_block = buffer.slice(slice_offset, slice_len); + + // Finally we check if all blocks emitted, if so, we reset the + // emit context to allow new updates + if self.emit_ctx.base_ctx.all_emitted() { + self.emit_ctx.base_ctx.reset(); + self.emit_ctx.buffer = None; + self.emit_ctx.last_block_len = 0; + } - let emit_block = emit_context.buffer.slice(slice_offset, slice_len); NullBuffer::new(emit_block) } + + fn size(&self) -> usize { + self.inner.size() + + size_of::() + + self + .emit_ctx + .buffer + .as_ref() + .map(|b| b.len() / 8) + .unwrap_or_default() + } } /// Invokes `value_fn(group_index, value)` for each non null, non