From eb742784da0aaf6cd6dd2207439868732ea05dfe Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 15 Aug 2024 00:24:51 +0800 Subject: [PATCH 001/107] re-design the sketch. --- .../expr-common/src/groups_accumulator.rs | 31 ++++- .../groups_accumulator/accumulate.rs | 11 +- .../aggregate/groups_accumulator/bool_op.rs | 9 +- .../aggregate/groups_accumulator/prim_op.rs | 2 +- datafusion/functions-aggregate/src/average.rs | 4 +- .../src/aggregates/group_values/bytes.rs | 6 + .../src/aggregates/group_values/bytes_view.rs | 6 + .../src/aggregates/group_values/mod.rs | 24 +++- .../src/aggregates/group_values/primitive.rs | 7 +- .../src/aggregates/group_values/row.rs | 5 + .../physical-plan/src/aggregates/row_hash.rs | 112 ++++++++++++++++-- 11 files changed, 196 insertions(+), 21 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index e66b27d073d1..ac1ec02c3f28 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)] @@ -31,6 +31,8 @@ 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 all groups managed by blocks + CurrentBlock(bool), } impl EmitTo { @@ -52,10 +54,17 @@ impl EmitTo { std::mem::swap(v, &mut t); t } + EmitTo::CurrentBlock(_) => unimplemented!(), } } } +#[derive(Debug, Clone, Copy)] +pub enum GroupStatesMode { + Flat, + Blocked(usize), +} + /// `GroupAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// @@ -143,6 +152,26 @@ pub trait GroupsAccumulator: Send { /// [`Accumulator::state`]: crate::accumulator::Accumulator::state fn state(&mut self, emit_to: EmitTo) -> Result>; + /// Returns `true` if blocked emission is supported + /// The blocked emission is possible to avoid result splitting in aggregation. + fn supports_blocked_emission(&self) -> bool { + false + } + + fn supports_blocked_mode(&self) -> bool { + false + } + + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + if matches!(&mode, GroupStatesMode::Blocked(_)) { + return Err(DataFusionError::NotImplemented( + "only flat mode is not supported yet".to_string(), + )); + } + + Ok(()) + } + /// Merges intermediate state (the output from [`Self::state`]) /// into this accumulator's current state. /// 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 455fc5fec450..ae5534fbcbcf 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,9 @@ use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; +use datafusion_common::{DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::EmitTo; + /// 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. /// @@ -329,7 +331,7 @@ impl NullState { /// for the `emit_to` rows. /// /// resets the internal state appropriately - pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { + pub fn build(&mut self, emit_to: EmitTo) -> Result { let nulls: BooleanBuffer = self.seen_values.finish(); let nulls = match emit_to { @@ -346,8 +348,13 @@ impl NullState { } first_n_null } + EmitTo::CurrentBlock(_) => { + return Err(DataFusionError::NotImplemented( + "blocked group values management is not supported".to_string(), + )) + } }; - NullBuffer::new(nulls) + Ok(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..7f38e7795b15 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::{DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use super::accumulate::NullState; @@ -117,9 +117,14 @@ where } first_n } + EmitTo::CurrentBlock(_) => { + return Err(DataFusionError::NotImplemented( + "blocked group values management is not supported".to_string(), + )) + } }; - let nulls = self.null_state.build(emit_to); + let nulls = self.null_state.build(emit_to)?; let values = BooleanArray::new(values, Some(nulls)); Ok(Arc::new(values)) } 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 b5c6171af37c..f1cec6c2bb0a 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 nulls = self.null_state.build(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()); Ok(Arc::new(values)) diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index ddad76a8734b..54bf163ecfe1 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -467,7 +467,7 @@ 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 nulls = self.null_state.build(emit_to); + let nulls = self.null_state.build(emit_to)?; assert_eq!(nulls.len(), sums.len()); assert_eq!(counts.len(), sums.len()); @@ -502,7 +502,7 @@ where // return arrays for sums and counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let nulls = self.null_state.build(emit_to); + let nulls = self.null_state.build(emit_to)?; let nulls = Some(nulls); let counts = emit_to.take_needed(&mut self.counts); diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs index f789af8b8a02..e2fb47722cb8 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs @@ -17,6 +17,7 @@ use crate::aggregates::group_values::GroupValues; use arrow_array::{Array, ArrayRef, OffsetSizeTrait, RecordBatch}; +use datafusion_common::DataFusionError; use datafusion_expr::EmitTo; use datafusion_physical_expr_common::binary_map::{ArrowBytesMap, OutputType}; @@ -115,6 +116,11 @@ impl GroupValues for GroupValuesByes { emit_group_values } + EmitTo::CurrentBlock(_) => { + return Err(DataFusionError::NotImplemented( + "blocked group values is not supported yet".to_string(), + )) + } }; Ok(vec![group_values]) diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs index 1a0cb90a16d4..82bdd76e43d0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs @@ -17,6 +17,7 @@ use crate::aggregates::group_values::GroupValues; use arrow_array::{Array, ArrayRef, RecordBatch}; +use datafusion_common::DataFusionError; use datafusion_expr::EmitTo; use datafusion_physical_expr::binary_map::OutputType; use datafusion_physical_expr_common::binary_view_map::ArrowBytesViewMap; @@ -116,6 +117,11 @@ impl GroupValues for GroupValuesBytesView { emit_group_values } + EmitTo::CurrentBlock(_) => { + return Err(DataFusionError::NotImplemented( + "blocked group values is not supported yet".to_string(), + )) + } }; Ok(vec![group_values]) diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index be7ac934d7bc..e644f75e50ef 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -19,10 +19,10 @@ use arrow::record_batch::RecordBatch; use arrow_array::{downcast_primitive, ArrayRef}; use arrow_schema::{DataType, SchemaRef}; use bytes_view::GroupValuesBytesView; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; pub(crate) mod primitive; -use datafusion_expr::EmitTo; +use datafusion_expr::{groups_accumulator::GroupStatesMode, EmitTo}; use primitive::GroupValuesPrimitive; mod row; @@ -36,7 +36,11 @@ use datafusion_physical_expr::binary_map::OutputType; /// An interning store for group keys pub trait GroupValues: Send { /// Calculates the `groups` for each input row of `cols` - fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()>; + fn intern( + &mut self, + cols: &[ArrayRef], + groups: &mut Vec, + ) -> Result<()>; /// Returns the number of bytes used by this [`GroupValues`] fn size(&self) -> usize; @@ -52,6 +56,20 @@ pub trait GroupValues: Send { /// Clear the contents and shrink the capacity to the size of the batch (free up memory usage) fn clear_shrink(&mut self, batch: &RecordBatch); + + fn supports_blocked_mode(&self) -> bool { + false + } + + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + if matches!(&mode, GroupStatesMode::Blocked(_)) { + return Err(DataFusionError::NotImplemented( + "only flat mode is not supported yet".to_string(), + )); + } + + Ok(()) + } } pub fn new_group_values(schema: SchemaRef) -> Result> { diff --git a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs index d5b7f1b11ac5..ba63c8eb5733 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs @@ -25,7 +25,7 @@ use arrow_array::cast::AsArray; use arrow_array::{ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, PrimitiveArray}; use arrow_buffer::{IntervalDayTime, IntervalMonthDayNano}; use arrow_schema::DataType; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_expr::EmitTo; use half::f16; @@ -206,6 +206,11 @@ where std::mem::swap(&mut self.values, &mut split); build_primitive(split, null_group) } + EmitTo::CurrentBlock(_) => { + return Err(DataFusionError::NotImplemented( + "blocked group values is not supported yet".to_string(), + )) + } }; Ok(vec![Arc::new(array.with_data_type(self.data_type.clone()))]) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index dc948e28bb2d..891e909c732b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -216,6 +216,11 @@ impl GroupValues for GroupValuesRows { } output } + EmitTo::CurrentBlock(_) => { + return Err(DataFusionError::NotImplemented( + "blocked group values management is not supported".to_string(), + )); + } }; // TODO: Materialize dictionaries in group keys (#7647) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 05f4ec621813..4cf0be9586ba 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -45,6 +45,7 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ @@ -62,6 +63,7 @@ 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), + ProducingBlocks(Option), /// Produce intermediate aggregate state for each input row without /// aggregation. /// @@ -431,6 +433,9 @@ pub(crate) struct GroupedHashAggregateStream { /// The [`RuntimeEnv`] associated with the [`TaskContext`] argument runtime: Arc, + enable_blocked_group_states: bool, + + group_states_size: usize, } impl GroupedHashAggregateStream { @@ -477,7 +482,7 @@ impl GroupedHashAggregateStream { }; // Instantiate the accumulators - let accumulators: Vec<_> = aggregate_exprs + let mut accumulators: Vec<_> = aggregate_exprs .iter() .map(create_group_accumulator) .collect::>()?; @@ -507,7 +512,7 @@ impl GroupedHashAggregateStream { ordering.as_slice(), )?; - let group_values = new_group_values(group_schema)?; + let mut group_values = new_group_values(group_schema)?; timer.done(); let exec_state = ExecutionState::ReadingInput; @@ -551,6 +556,14 @@ impl GroupedHashAggregateStream { None }; + // Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. + let enable_blocked_group_states = maybe_enable_blocked_group_states( + group_values.as_mut(), + &mut accumulators, + batch_size, + &group_ordering, + )?; + Ok(GroupedHashAggregateStream { schema: agg_schema, input, @@ -571,10 +584,40 @@ impl GroupedHashAggregateStream { spill_state, group_values_soft_limit: agg.limit, skip_aggregation_probe, + enable_blocked_group_states, + group_states_size: batch_size, }) } } +/// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. +/// The blocked optimization will be enabled when: +/// - It is not streaming aggregation(because blocked mode can't support Emit::first(exact n)) +/// - The accumulator is not empty +/// - `GroupValues` and all `GroupsAccumulator`s support blocked mode +fn maybe_enable_blocked_group_states( + group_values: &mut dyn GroupValues, + accumulators: &mut [Box], + block_size: usize, + group_ordering: &GroupOrdering, +) -> Result { + if matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() { + return Ok(false); + } + + let group_supports_blocked = group_values.supports_blocked_mode(); + let accumulators_support_blocked = + accumulators.iter().any(|acc| acc.supports_blocked_mode()); + if group_supports_blocked && accumulators_support_blocked { + group_values.switch_to_mode(GroupStatesMode::Blocked(block_size))?; + accumulators + .iter_mut() + .try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Blocked(block_size)))?; + } + + Ok(true) +} + /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. @@ -720,6 +763,44 @@ impl Stream for GroupedHashAggregateStream { ))); } + ExecutionState::ProducingBlocks(blocks) => { + if let Some(blk) = blocks { + if *blk > 0 { + self.exec_state = + ExecutionState::ProducingBlocks(Some(*blk - 1)); + } else { + self.exec_state = if self.input_done { + ExecutionState::Done + } else if self.should_skip_aggregation() { + ExecutionState::SkippingAggregation + } else { + ExecutionState::ReadingInput + }; + continue; + } + } + + let emit_result = self.emit(EmitTo::CurrentBlock(true), false); + if emit_result.is_err() { + return Poll::Ready(Some(emit_result)); + } + + let emit_batch = emit_result.unwrap(); + if emit_batch.num_rows() == 0 { + self.exec_state = if self.input_done { + ExecutionState::Done + } else if self.should_skip_aggregation() { + ExecutionState::SkippingAggregation + } else { + ExecutionState::ReadingInput + }; + } + + return Poll::Ready(Some(Ok( + emit_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. @@ -945,9 +1026,14 @@ impl GroupedHashAggregateStream { && matches!(self.mode, AggregateMode::Partial) && self.update_memory_reservation().is_err() { - let n = self.group_values.len() / self.batch_size * self.batch_size; - let batch = self.emit(EmitTo::First(n), false)?; - self.exec_state = ExecutionState::ProducingOutput(batch); + if self.enable_blocked_group_states { + let n = self.group_values.len() / self.batch_size * self.batch_size; + let batch = self.emit(EmitTo::First(n), false)?; + self.exec_state = ExecutionState::ProducingOutput(batch); + } else { + let blocks = self.group_values.len() / self.group_states_size; + self.exec_state = ExecutionState::ProducingBlocks(Some(blocks)); + } } Ok(()) } @@ -1005,8 +1091,12 @@ 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)?; - ExecutionState::ProducingOutput(batch) + if !self.enable_blocked_group_states { + let batch = self.emit(EmitTo::All, false)?; + ExecutionState::ProducingOutput(batch) + } else { + ExecutionState::ProducingBlocks(None) + } } else { // If spill files exist, stream-merge them. self.update_merged_stream()?; @@ -1038,8 +1128,12 @@ impl GroupedHashAggregateStream { fn switch_to_skip_aggregation(&mut self) -> Result<()> { if let Some(probe) = self.skip_aggregation_probe.as_mut() { if probe.should_skip() { - let batch = self.emit(EmitTo::All, false)?; - self.exec_state = ExecutionState::ProducingOutput(batch); + if !self.enable_blocked_group_states { + let batch = self.emit(EmitTo::All, false)?; + self.exec_state = ExecutionState::ProducingOutput(batch); + } else { + self.exec_state = ExecutionState::ProducingBlocks(None); + } } } From 728a62e4ef6d302eaaf5747b6314485e784e4156 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 15 Aug 2024 17:15:29 +0800 Subject: [PATCH 002/107] disable blocked optimization when hash agg is swtched to streaming agg due to spilling. --- .../groups_accumulator/accumulate.rs | 4 +-- .../src/aggregates/group_values/row.rs | 2 +- .../physical-plan/src/aggregates/row_hash.rs | 36 +++++++++++++------ 3 files changed, 28 insertions(+), 14 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 ae5534fbcbcf..3cb3088288ce 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -719,7 +719,7 @@ mod test { // 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(EmitTo::All).unwrap(); assert_eq!(null_buffer, expected_null_buffer); } @@ -836,7 +836,7 @@ mod test { // 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(EmitTo::All).unwrap(); assert_eq!(null_buffer, expected_null_buffer); } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 891e909c732b..50804159ae6e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -218,7 +218,7 @@ impl GroupValues for GroupValuesRows { } EmitTo::CurrentBlock(_) => { return Err(DataFusionError::NotImplemented( - "blocked group values management is not supported".to_string(), + "blocked group values is not supported yet".to_string(), )); } }; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 4cf0be9586ba..e768c1537c21 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -435,7 +435,7 @@ pub(crate) struct GroupedHashAggregateStream { runtime: Arc, enable_blocked_group_states: bool, - group_states_size: usize, + group_states_block_size: usize, } impl GroupedHashAggregateStream { @@ -563,6 +563,7 @@ impl GroupedHashAggregateStream { batch_size, &group_ordering, )?; + dbg!(enable_blocked_group_states); Ok(GroupedHashAggregateStream { schema: agg_schema, @@ -585,7 +586,7 @@ impl GroupedHashAggregateStream { group_values_soft_limit: agg.limit, skip_aggregation_probe, enable_blocked_group_states, - group_states_size: batch_size, + group_states_block_size: batch_size, }) } } @@ -608,14 +609,17 @@ fn maybe_enable_blocked_group_states( let group_supports_blocked = group_values.supports_blocked_mode(); let accumulators_support_blocked = accumulators.iter().any(|acc| acc.supports_blocked_mode()); - if group_supports_blocked && accumulators_support_blocked { - group_values.switch_to_mode(GroupStatesMode::Blocked(block_size))?; - accumulators - .iter_mut() - .try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Blocked(block_size)))?; - } - Ok(true) + match (group_supports_blocked, accumulators_support_blocked) { + (true, true) => { + group_values.switch_to_mode(GroupStatesMode::Blocked(block_size))?; + accumulators + .iter_mut() + .try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Blocked(block_size)))?; + Ok(true) + } + _ => Ok(false) + } } /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if @@ -1026,12 +1030,12 @@ impl GroupedHashAggregateStream { && matches!(self.mode, AggregateMode::Partial) && self.update_memory_reservation().is_err() { - if self.enable_blocked_group_states { + if !self.enable_blocked_group_states { let n = self.group_values.len() / self.batch_size * self.batch_size; let batch = self.emit(EmitTo::First(n), false)?; self.exec_state = ExecutionState::ProducingOutput(batch); } else { - let blocks = self.group_values.len() / self.group_states_size; + let blocks = self.group_values.len() / self.group_states_block_size; self.exec_state = ExecutionState::ProducingBlocks(Some(blocks)); } } @@ -1070,6 +1074,16 @@ impl GroupedHashAggregateStream { None, self.reservation.new_empty(), )?; + + // We should disable the blocked optimization for `GroupValues` and `GroupAccumulator`s here, + // because the blocked mode can't support `Emit::First(exact n)` which is needed in + // streaming aggregation. + if self.enable_blocked_group_states { + self.group_values.switch_to_mode(GroupStatesMode::Flat)?; + self.accumulators.iter_mut().try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Flat))?; + self.enable_blocked_group_states = false; + } + self.input_done = false; self.group_ordering = GroupOrdering::Full(GroupOrderingFull::new()); Ok(()) From a1f5e2d6b284b57a5434410187c6785d3e6bf398 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 15 Aug 2024 17:23:44 +0800 Subject: [PATCH 003/107] fix style. --- .../src/aggregates/group_values/mod.rs | 6 +----- .../physical-plan/src/aggregates/row_hash.rs | 16 +++++++++------- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index e644f75e50ef..208761f6e35f 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -36,11 +36,7 @@ use datafusion_physical_expr::binary_map::OutputType; /// An interning store for group keys pub trait GroupValues: Send { /// Calculates the `groups` for each input row of `cols` - fn intern( - &mut self, - cols: &[ArrayRef], - groups: &mut Vec, - ) -> Result<()>; + fn intern(&mut self, cols: &[ArrayRef], groups: &mut Vec) -> Result<()>; /// Returns the number of bytes used by this [`GroupValues`] fn size(&self) -> usize; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index e768c1537c21..264b7e27d173 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -613,12 +613,12 @@ fn maybe_enable_blocked_group_states( match (group_supports_blocked, accumulators_support_blocked) { (true, true) => { group_values.switch_to_mode(GroupStatesMode::Blocked(block_size))?; - accumulators - .iter_mut() - .try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Blocked(block_size)))?; + accumulators.iter_mut().try_for_each(|acc| { + acc.switch_to_mode(GroupStatesMode::Blocked(block_size)) + })?; Ok(true) } - _ => Ok(false) + _ => Ok(false), } } @@ -1076,14 +1076,16 @@ impl GroupedHashAggregateStream { )?; // We should disable the blocked optimization for `GroupValues` and `GroupAccumulator`s here, - // because the blocked mode can't support `Emit::First(exact n)` which is needed in + // because the blocked mode can't support `Emit::First(exact n)` which is needed in // streaming aggregation. if self.enable_blocked_group_states { self.group_values.switch_to_mode(GroupStatesMode::Flat)?; - self.accumulators.iter_mut().try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Flat))?; + self.accumulators + .iter_mut() + .try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Flat))?; self.enable_blocked_group_states = false; } - + self.input_done = false; self.group_ordering = GroupOrdering::Full(GroupOrderingFull::new()); Ok(()) From d7d22f69e963c23a5c4065ab22384e5de8d9c6c6 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 15 Aug 2024 22:02:53 +0800 Subject: [PATCH 004/107] impl blocked GroupValuesRows. --- .../src/aggregates/group_values/row.rs | 205 ++++++++++++++---- 1 file changed, 166 insertions(+), 39 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 50804159ae6e..4af23897322e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -15,6 +15,9 @@ // specific language governing permissions and limitations // under the License. +use std::collections::VecDeque; +use std::mem; + use crate::aggregates::group_values::GroupValues; use ahash::RandomState; use arrow::compute::cast; @@ -25,6 +28,7 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::EmitTo; use hashbrown::raw::RawTable; @@ -57,7 +61,7 @@ pub struct GroupValuesRows { /// important for multi-column group keys. /// /// [`Row`]: arrow::row::Row - group_values: Option, + group_values: VecDeque, /// reused buffer to store hashes hashes_buffer: Vec, @@ -67,6 +71,9 @@ pub struct GroupValuesRows { /// Random state for creating hashes random_state: RandomState, + + /// Mode about current GroupValuesRows + mode: GroupStatesMode, } impl GroupValuesRows { @@ -90,10 +97,11 @@ impl GroupValuesRows { row_converter, map, map_size: 0, - group_values: None, + group_values: VecDeque::new(), hashes_buffer: Default::default(), rows_buffer, random_state: Default::default(), + mode: GroupStatesMode::Flat, }) } } @@ -106,10 +114,17 @@ impl GroupValues for GroupValuesRows { self.row_converter.append(group_rows, cols)?; let n_rows = group_rows.num_rows(); - let mut group_values = match self.group_values.take() { - Some(group_values) => group_values, - None => self.row_converter.empty_rows(0, 0), - }; + let mut group_values = mem::take(&mut self.group_values); + if group_values.is_empty() { + let block = match self.mode { + GroupStatesMode::Flat => self.row_converter.empty_rows(0, 0), + GroupStatesMode::Blocked(blk_size) => { + self.row_converter.empty_rows(blk_size, 0) + } + }; + + group_values.push_back(block); + } // tracks to which group each of the input rows belongs groups.clear(); @@ -121,16 +136,32 @@ impl GroupValues for GroupValuesRows { create_hashes(cols, &self.random_state, batch_hashes)?; for (row, &target_hash) in batch_hashes.iter().enumerate() { + let mode = self.mode; let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { // Somewhat surprisingly, this closure can be called even if the // hash doesn't match, so check the hash first with an integer // comparison first avoid the more expensive comparison with // group value. https://github.com/apache/datafusion/pull/11718 - target_hash == *exist_hash - // verify that the group that we are inserting with hash is - // actually the same key value as the group in - // existing_idx (aka group_values @ row) - && group_rows.row(row) == group_values.row(*group_idx) + if target_hash != *exist_hash { + return false; + } + + // verify that the group that we are inserting with hash is + // actually the same key value as the group in + // existing_idx (aka group_values @ row) + match mode { + GroupStatesMode::Flat => { + group_rows.row(row) + == group_values.back().unwrap().row(*group_idx) + } + GroupStatesMode::Blocked(_) => { + let block_id = + ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; + let block_offset = + ((*group_idx as u64) & 0x00000000ffffffff) as usize; + group_rows.row(row) == group_values[block_id].row(block_offset) + } + } }); let group_idx = match entry { @@ -139,8 +170,34 @@ impl GroupValues for GroupValuesRows { // 1.2 Need to create new entry for the group None => { // Add new entry to aggr_state and save newly created index - let group_idx = group_values.num_rows(); - group_values.push(group_rows.row(row)); + let group_idx = match mode { + GroupStatesMode::Flat => { + let blk = group_values.back_mut().unwrap(); + let group_idx = blk.num_rows(); + blk.push(group_rows.row(row)); + group_idx + } + GroupStatesMode::Blocked(blk_size) => { + if group_values.back().unwrap().num_rows() == blk_size { + // Use blk_size as offset cap, + // and old block's buffer size as buffer cap + let new_buf_cap = + rows_buffer_size(group_values.back().unwrap()); + let new_blk = + self.row_converter.empty_rows(blk_size, new_buf_cap); + group_values.push_back(new_blk); + } + + let blk_id = (group_values.len() - 1) as u64; + let cur_blk = group_values.back_mut().unwrap(); + let blk_offset = cur_blk.num_rows() as u64; + cur_blk.push(group_rows.row(row)); + + (((blk_id << 32) & 0xffffffff00000000) + | (blk_offset & 0x00000000ffffffff)) + as usize + } + }; // for hasher function, use precomputed hash value self.map.insert_accounted( @@ -154,13 +211,17 @@ impl GroupValues for GroupValuesRows { groups.push(group_idx); } - self.group_values = Some(group_values); + self.group_values = group_values; Ok(()) } fn size(&self) -> usize { - let group_values_size = self.group_values.as_ref().map(|v| v.size()).unwrap_or(0); + let group_values_size = self + .group_values + .iter() + .map(|rows| rows.size()) + .sum::(); self.row_converter.size() + group_values_size + self.map_size @@ -174,33 +235,58 @@ impl GroupValues for GroupValuesRows { fn len(&self) -> usize { self.group_values - .as_ref() - .map(|group_values| group_values.num_rows()) - .unwrap_or(0) + .iter() + .map(|rows| rows.num_rows()) + .sum::() } fn emit(&mut self, emit_to: EmitTo) -> Result> { - let mut group_values = self - .group_values - .take() - .expect("Can not emit from empty rows"); + let mut group_values = mem::take(&mut self.group_values); let mut output = match emit_to { - EmitTo::All => { - let output = self.row_converter.convert_rows(&group_values)?; - group_values.clear(); - output - } + EmitTo::All => match self.mode { + GroupStatesMode::Flat => { + let blk = group_values.pop_back().unwrap(); + let output = self.row_converter.convert_rows(blk.into_iter())?; + output + } + GroupStatesMode::Blocked(_) => { + let mut total_rows_num = 0; + let mut total_buffer_size = 0; + group_values.iter().for_each(|rows| { + let rows_num = rows.num_rows(); + let rows_buffer_size = rows_buffer_size(rows); + total_rows_num += rows_num; + total_buffer_size += rows_buffer_size; + }); + + let mut total_rows = self + .row_converter + .empty_rows(total_rows_num, total_buffer_size); + for rows in &group_values { + for row in rows.into_iter() { + total_rows.push(row); + } + } + + group_values.clear(); + + let output = + self.row_converter.convert_rows(total_rows.into_iter())?; + output + } + }, EmitTo::First(n) => { - let groups_rows = group_values.iter().take(n); + let blk = group_values.back_mut().unwrap(); + let groups_rows = blk.iter().take(n); let output = self.row_converter.convert_rows(groups_rows)?; // Clear out first n group keys by copying them to a new Rows. // TODO file some ticket in arrow-rs to make this more efficient? let mut new_group_values = self.row_converter.empty_rows(0, 0); - for row in group_values.iter().skip(n) { + for row in blk.iter().skip(n) { new_group_values.push(row); } - std::mem::swap(&mut new_group_values, &mut group_values); + std::mem::swap(&mut new_group_values, blk); // SAFETY: self.map outlives iterator and is not modified concurrently unsafe { @@ -216,10 +302,34 @@ impl GroupValues for GroupValuesRows { } output } - EmitTo::CurrentBlock(_) => { - return Err(DataFusionError::NotImplemented( - "blocked group values is not supported yet".to_string(), - )); + EmitTo::CurrentBlock(true) => { + let cur_blk = group_values.pop_front().unwrap(); + let output = self.row_converter.convert_rows(cur_blk.iter())?; + unsafe { + for bucket in self.map.iter() { + // Decrement group index by n + let group_idx = bucket.as_ref().1 as u64; + let blk_id = (group_idx >> 32) & 0x00000000ffffffff; + let blk_offset = group_idx & 0x00000000ffffffff; + match blk_id.checked_sub(1) { + // Group index was >= n, shift value down + Some(bid) => { + let new_group_idx = (((bid << 32) & 0xffffffff00000000) + | (blk_offset & 0x00000000ffffffff)) + as usize; + bucket.as_mut().1 = new_group_idx; + } + // Group index was < n, so remove from table + None => self.map.erase(bucket), + } + } + } + output + } + EmitTo::CurrentBlock(false) => { + let cur_blk = group_values.pop_front().unwrap(); + let output = self.row_converter.convert_rows(cur_blk.iter())?; + output } }; @@ -237,20 +347,37 @@ impl GroupValues for GroupValuesRows { } } - self.group_values = Some(group_values); + self.group_values = group_values; + Ok(output) } fn clear_shrink(&mut self, batch: &RecordBatch) { let count = batch.num_rows(); - self.group_values = self.group_values.take().map(|mut rows| { - rows.clear(); - rows - }); + self.group_values.clear(); self.map.clear(); self.map.shrink_to(count, |_| 0); // hasher does not matter since the map is cleared self.map_size = self.map.capacity() * std::mem::size_of::<(u64, usize)>(); self.hashes_buffer.clear(); self.hashes_buffer.shrink_to(count); } + + fn supports_blocked_mode(&self) -> bool { + true + } + + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + self.map.clear(); + self.group_values.clear(); + self.mode = mode; + + Ok(()) + } +} + +#[inline] +fn rows_buffer_size(rows: &Rows) -> usize { + let total_size = rows.size(); + let offset_size = (rows.num_rows() + 1) * mem::size_of::(); + total_size - offset_size - mem::size_of::() } From e62172a9a945c798a83ec6b5ba3636a881debe92 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 15 Aug 2024 23:02:24 +0800 Subject: [PATCH 005/107] impl simple blocked mode for Count. --- .../expr-common/src/groups_accumulator.rs | 17 +++--- .../src/aggregate/groups_accumulator.rs | 4 +- datafusion/functions-aggregate/src/count.rs | 57 +++++++++++++++++-- 3 files changed, 63 insertions(+), 15 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index ac1ec02c3f28..514545a6584b 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -41,7 +41,7 @@ impl EmitTo { /// remaining values in `v`. /// /// This avoids copying if Self::All - pub fn take_needed(&self, v: &mut Vec) -> Vec { + pub fn take_needed(&self, v: &mut Vec, block_size: Option) -> Vec { match self { Self::All => { // Take the entire vector, leave new (empty) vector @@ -54,7 +54,14 @@ impl EmitTo { std::mem::swap(v, &mut t); t } - EmitTo::CurrentBlock(_) => unimplemented!(), + EmitTo::CurrentBlock(_) => { + let block_size = block_size.unwrap(); + // get end n+1,.. values into t + let mut t = v.split_off(block_size); + // leave n+1,.. in v + std::mem::swap(v, &mut t); + t + } } } } @@ -152,12 +159,6 @@ pub trait GroupsAccumulator: Send { /// [`Accumulator::state`]: crate::accumulator::Accumulator::state fn state(&mut self, emit_to: EmitTo) -> Result>; - /// Returns `true` if blocked emission is supported - /// The blocked emission is possible to avoid result splitting in aggregation. - fn supports_blocked_emission(&self) -> bool { - false - } - fn supports_blocked_mode(&self) -> bool { false } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 1c97d22ec79c..b6218a50983c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -273,7 +273,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(&mut self.states, None); let results: Vec = states .into_iter() @@ -292,7 +292,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { 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(&mut self.states, None); // each accumulator produces a potential vector of values // which we need to form into columns diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 417e28e72a71..8254945267f0 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,8 +16,9 @@ // under the License. use ahash::RandomState; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; -use std::collections::HashSet; +use std::collections::{HashSet, VecDeque}; use std::ops::BitAnd; use std::{fmt::Debug, sync::Arc}; @@ -358,12 +359,20 @@ struct CountGroupsAccumulator { /// output type of count is `DataType::Int64`. Thus by using `i64` /// for the counts, the output [`Int64Array`] can be created /// without copy. - counts: Vec, + counts: VecDeque, + + mode: GroupStatesMode, + + group_idx_convert_buffer: Vec, } impl CountGroupsAccumulator { pub fn new() -> Self { - Self { counts: vec![] } + Self { + counts: VecDeque::new(), + mode: GroupStatesMode::Flat, + group_idx_convert_buffer: Vec::new(), + } } } @@ -381,6 +390,25 @@ impl GroupsAccumulator for CountGroupsAccumulator { // Add one to each group's counter for each non null, non // filtered value self.counts.resize(total_num_groups, 0); + + let group_indices = match self.mode { + GroupStatesMode::Flat => group_indices, + GroupStatesMode::Blocked(blk_size) => { + self.group_idx_convert_buffer.clear(); + + let converted_group_indices = group_indices.iter().map(|group_idx| { + let blk_id = + ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; + let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; + blk_id * blk_size + blk_offset + }); + self.group_idx_convert_buffer + .extend(converted_group_indices); + + &self.group_idx_convert_buffer + } + }; + accumulate_indices( group_indices, values.logical_nulls().as_ref(), @@ -431,7 +459,11 @@ impl GroupsAccumulator for CountGroupsAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed(&mut self.counts); + let block_size = match self.mode { + GroupStatesMode::Flat => None, + GroupStatesMode::Blocked(blk_size) => Some(blk_size), + }; + let counts = emit_to.take_needed(&mut self.counts, block_size); // Count is always non null (null inputs just don't contribute to the overall values) let nulls = None; @@ -442,8 +474,13 @@ 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 block_size = match self.mode { + GroupStatesMode::Flat => None, + GroupStatesMode::Blocked(blk_size) => Some(blk_size), + }; + let counts = emit_to.take_needed(&mut self.counts, block_size); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls + Ok(vec![Arc::new(counts) as ArrayRef]) } @@ -515,6 +552,16 @@ impl GroupsAccumulator for CountGroupsAccumulator { fn size(&self) -> usize { self.counts.capacity() * std::mem::size_of::() } + + fn supports_blocked_mode(&self) -> bool { + true + } + + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + self.counts.clear(); + self.mode = mode; + Ok(()) + } } /// count null values for multiple columns From 99cd66a5c26dab422eb8e7402fb141450dc1cd21 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 15 Aug 2024 23:21:58 +0800 Subject: [PATCH 006/107] impl simple blocked mod for prim_op and avg. --- .../groups_accumulator/accumulate.rs | 28 +++++++--- .../aggregate/groups_accumulator/bool_op.rs | 2 +- .../aggregate/groups_accumulator/prim_op.rs | 52 +++++++++++++++++-- datafusion/functions-aggregate/src/average.rs | 40 +++++++++++--- datafusion/functions-aggregate/src/count.rs | 10 ++-- .../src/aggregates/group_values/row.rs | 8 +-- 6 files changed, 115 insertions(+), 25 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 3cb3088288ce..ac23710ecd3b 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 arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::Result; use datafusion_expr_common::groups_accumulator::EmitTo; /// Track the accumulator null state per row: if any values for that @@ -331,7 +331,12 @@ impl NullState { /// for the `emit_to` rows. /// /// resets the internal state appropriately - pub fn build(&mut self, emit_to: EmitTo) -> Result { + // TODO: add unit tests for the `EmitTo::CurrentBlock` branch + pub fn build( + &mut self, + emit_to: EmitTo, + block_size: Option, + ) -> Result { let nulls: BooleanBuffer = self.seen_values.finish(); let nulls = match emit_to { @@ -349,9 +354,18 @@ impl NullState { first_n_null } EmitTo::CurrentBlock(_) => { - return Err(DataFusionError::NotImplemented( - "blocked group values management is not supported".to_string(), - )) + let block_size = block_size.unwrap(); + + // 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(block_size).collect(); + // reset the existing seen buffer + for seen in nulls.iter().skip(block_size) { + self.seen_values.append(seen); + } + first_n_null } }; Ok(NullBuffer::new(nulls)) @@ -719,7 +733,7 @@ mod test { // 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).unwrap(); + let null_buffer = null_state.build(EmitTo::All, None).unwrap(); assert_eq!(null_buffer, expected_null_buffer); } @@ -836,7 +850,7 @@ mod test { // 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).unwrap(); + let null_buffer = null_state.build(EmitTo::All, None).unwrap(); assert_eq!(null_buffer, expected_null_buffer); } 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 7f38e7795b15..7ffc795ebe99 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 @@ -124,7 +124,7 @@ where } }; - let nulls = self.null_state.build(emit_to)?; + let nulls = self.null_state.build(emit_to, None)?; let values = BooleanArray::new(values, Some(nulls)); Ok(Arc::new(values)) } 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 f1cec6c2bb0a..eb9897e3f06e 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 @@ -23,7 +23,9 @@ use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; -use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; +use datafusion_expr_common::groups_accumulator::{ + EmitTo, GroupStatesMode, GroupsAccumulator, +}; use super::accumulate::NullState; @@ -56,6 +58,10 @@ where /// Function that computes the primitive result prim_fn: F, + + mode: GroupStatesMode, + + group_idx_convert_buffer: Vec, } impl PrimitiveGroupsAccumulator @@ -70,6 +76,8 @@ where null_state: NullState::new(), starting_value: T::default_value(), prim_fn, + mode: GroupStatesMode::Flat, + group_idx_convert_buffer: Vec::new(), } } @@ -98,6 +106,25 @@ where // update values self.values.resize(total_num_groups, self.starting_value); + // Maybe we should convert the `group_indices` + let group_indices = match self.mode { + GroupStatesMode::Flat => group_indices, + GroupStatesMode::Blocked(blk_size) => { + self.group_idx_convert_buffer.clear(); + + let converted_group_indices = group_indices.iter().map(|group_idx| { + let blk_id = + ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; + let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; + blk_id * blk_size + blk_offset + }); + self.group_idx_convert_buffer + .extend(converted_group_indices); + + &self.group_idx_convert_buffer + } + }; + // NullState dispatches / handles tracking nulls and groups that saw no values self.null_state.accumulate( group_indices, @@ -114,10 +141,16 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = emit_to.take_needed(&mut self.values); - let nulls = self.null_state.build(emit_to)?; + let block_size = match self.mode { + GroupStatesMode::Flat => None, + GroupStatesMode::Blocked(blk_size) => Some(blk_size), + }; + + let values = emit_to.take_needed(&mut self.values, block_size); + let nulls = self.null_state.build(emit_to, block_size)?; let values = PrimitiveArray::::new(values.into(), Some(nulls)) // no copy .with_data_type(self.data_type.clone()); + Ok(Arc::new(values)) } @@ -197,4 +230,17 @@ where fn size(&self) -> usize { self.values.capacity() * std::mem::size_of::() + self.null_state.size() } + + fn supports_blocked_mode(&self) -> bool { + true + } + + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + self.values.clear(); + self.null_state = NullState::new(); + + self.mode = mode; + + Ok(()) + } } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 54bf163ecfe1..d80c9e88d882 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,6 +29,7 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; @@ -406,6 +407,8 @@ where /// Function that computes the final average (value / count) avg_fn: F, + + mode: GroupStatesMode, } impl AvgGroupsAccumulator @@ -426,6 +429,7 @@ where sums: vec![], null_state: NullState::new(), avg_fn, + mode: GroupStatesMode::Flat, } } } @@ -465,9 +469,14 @@ 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 nulls = self.null_state.build(emit_to)?; + let block_size = match self.mode { + GroupStatesMode::Flat => None, + GroupStatesMode::Blocked(blk_size) => Some(blk_size), + }; + + let counts = emit_to.take_needed(&mut self.counts, block_size); + let sums = emit_to.take_needed(&mut self.sums, block_size); + let nulls = self.null_state.build(emit_to, block_size)?; assert_eq!(nulls.len(), sums.len()); assert_eq!(counts.len(), sums.len()); @@ -502,13 +511,18 @@ where // return arrays for sums and counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let nulls = self.null_state.build(emit_to)?; + let block_size = match self.mode { + GroupStatesMode::Flat => None, + GroupStatesMode::Blocked(blk_size) => Some(blk_size), + }; + + let nulls = self.null_state.build(emit_to, block_size)?; let nulls = Some(nulls); - let counts = emit_to.take_needed(&mut self.counts); + let counts = emit_to.take_needed(&mut self.counts, block_size); 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(&mut self.sums, block_size); let sums = PrimitiveArray::::new(sums.into(), nulls) // zero copy .with_data_type(self.sum_data_type.clone()); @@ -585,4 +599,18 @@ where self.counts.capacity() * std::mem::size_of::() + self.sums.capacity() * std::mem::size_of::() } + + fn supports_blocked_mode(&self) -> bool { + true + } + + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + self.counts.clear(); + self.sums.clear(); + self.null_state = NullState::new(); + + self.mode = mode; + + Ok(()) + } } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 8254945267f0..0f87e62353cb 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -18,7 +18,7 @@ use ahash::RandomState; use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; -use std::collections::{HashSet, VecDeque}; +use std::collections::HashSet; use std::ops::BitAnd; use std::{fmt::Debug, sync::Arc}; @@ -359,7 +359,7 @@ struct CountGroupsAccumulator { /// output type of count is `DataType::Int64`. Thus by using `i64` /// for the counts, the output [`Int64Array`] can be created /// without copy. - counts: VecDeque, + counts: Vec, mode: GroupStatesMode, @@ -369,7 +369,7 @@ struct CountGroupsAccumulator { impl CountGroupsAccumulator { pub fn new() -> Self { Self { - counts: VecDeque::new(), + counts: vec![], mode: GroupStatesMode::Flat, group_idx_convert_buffer: Vec::new(), } @@ -391,6 +391,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { // filtered value self.counts.resize(total_num_groups, 0); + // Maybe we should convert the `group_indices` let group_indices = match self.mode { GroupStatesMode::Flat => group_indices, GroupStatesMode::Blocked(blk_size) => { @@ -480,7 +481,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { }; let counts = emit_to.take_needed(&mut self.counts, block_size); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls - + Ok(vec![Arc::new(counts) as ArrayRef]) } @@ -560,6 +561,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { self.counts.clear(); self.mode = mode; + Ok(()) } } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 4af23897322e..c244ad414ad6 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -361,18 +361,18 @@ impl GroupValues for GroupValuesRows { self.hashes_buffer.clear(); self.hashes_buffer.shrink_to(count); } - + fn supports_blocked_mode(&self) -> bool { true } - + fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { self.map.clear(); self.group_values.clear(); self.mode = mode; - + Ok(()) - } + } } #[inline] From 520b2eb1c5ecde6c02453c2af071fc1d4344c412 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 16 Aug 2024 00:12:04 +0800 Subject: [PATCH 007/107] fix init. --- datafusion/expr-common/src/groups_accumulator.rs | 10 ++++++++-- .../physical-plan/src/aggregates/group_values/row.rs | 4 ++++ datafusion/physical-plan/src/aggregates/row_hash.rs | 3 +-- 3 files changed, 13 insertions(+), 4 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 514545a6584b..6ed1af6160c2 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::cmp::min; + use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -48,16 +50,20 @@ impl EmitTo { std::mem::take(v) } Self::First(n) => { + let split_at = min(v.len(), *n); + // get end n+1,.. values into t - let mut t = v.split_off(*n); + let mut t = v.split_off(split_at); // leave n+1,.. in v std::mem::swap(v, &mut t); t } EmitTo::CurrentBlock(_) => { let block_size = block_size.unwrap(); + let split_at = min(v.len(), block_size); + // get end n+1,.. values into t - let mut t = v.split_off(block_size); + let mut t = v.split_off(split_at); // leave n+1,.. in v std::mem::swap(v, &mut t); t diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index c244ad414ad6..e91da9d1504b 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -277,6 +277,10 @@ impl GroupValues for GroupValuesRows { } }, EmitTo::First(n) => { + if matches!(self.mode, GroupStatesMode::Blocked(_)) { + panic!("kamille debug"); + } + let blk = group_values.back_mut().unwrap(); let groups_rows = blk.iter().take(n); let output = self.row_converter.convert_rows(groups_rows)?; diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 264b7e27d173..79b7b79792e6 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -563,7 +563,6 @@ impl GroupedHashAggregateStream { batch_size, &group_ordering, )?; - dbg!(enable_blocked_group_states); Ok(GroupedHashAggregateStream { schema: agg_schema, @@ -602,7 +601,7 @@ fn maybe_enable_blocked_group_states( block_size: usize, group_ordering: &GroupOrdering, ) -> Result { - if matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() { + if !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() { return Ok(false); } From aae2a3be951f95a4b1630b18922e1e1b0cab84ba Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 16 Aug 2024 01:32:19 +0800 Subject: [PATCH 008/107] fix tests. --- .../expr-common/src/groups_accumulator.rs | 8 +++- .../aggregate/groups_accumulator/prim_op.rs | 6 +-- datafusion/functions-aggregate/src/average.rs | 43 +++++++++++++++++++ datafusion/functions-aggregate/src/count.rs | 28 ++++++++++-- .../src/aggregates/group_values/mod.rs | 2 +- .../physical-plan/src/aggregates/row_hash.rs | 12 ++++-- 6 files changed, 85 insertions(+), 14 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 6ed1af6160c2..952721d38898 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -51,7 +51,7 @@ impl EmitTo { } Self::First(n) => { let split_at = min(v.len(), *n); - + // get end n+1,.. values into t let mut t = v.split_off(split_at); // leave n+1,.. in v @@ -171,8 +171,12 @@ pub trait GroupsAccumulator: Send { fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { + // if self.supports_blocked_mode() { + // panic!("kamille debug debug"); + // } + return Err(DataFusionError::NotImplemented( - "only flat mode is not supported yet".to_string(), + "only flat mode accumulator is supported yet".to_string(), )); } 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 eb9897e3f06e..c8f891b5ea00 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 @@ -103,9 +103,6 @@ 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); - // Maybe we should convert the `group_indices` let group_indices = match self.mode { GroupStatesMode::Flat => group_indices, @@ -125,6 +122,9 @@ where } }; + // update values + self.values.resize(total_num_groups, self.starting_value); + // NullState dispatches / handles tracking nulls and groups that saw no values self.null_state.accumulate( group_indices, diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index d80c9e88d882..7797c051e789 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -409,6 +409,8 @@ where avg_fn: F, mode: GroupStatesMode, + + group_idx_convert_buffer: Vec, } impl AvgGroupsAccumulator @@ -430,6 +432,7 @@ where null_state: NullState::new(), avg_fn, mode: GroupStatesMode::Flat, + group_idx_convert_buffer: Vec::new(), } } } @@ -449,9 +452,29 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); + // Maybe we should convert the `group_indices` + let group_indices = match self.mode { + GroupStatesMode::Flat => group_indices, + GroupStatesMode::Blocked(blk_size) => { + self.group_idx_convert_buffer.clear(); + + let converted_group_indices = group_indices.iter().map(|group_idx| { + let blk_id = + ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; + let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; + blk_id * blk_size + blk_offset + }); + self.group_idx_convert_buffer + .extend(converted_group_indices); + + &self.group_idx_convert_buffer + } + }; + // increment counts, update sums self.counts.resize(total_num_groups, 0); self.sums.resize(total_num_groups, T::default_value()); + self.null_state.accumulate( group_indices, values, @@ -543,6 +566,26 @@ where // first batch is counts, second is partial sums let partial_counts = values[0].as_primitive::(); let partial_sums = values[1].as_primitive::(); + + // Maybe we should convert the `group_indices` + let group_indices = match self.mode { + GroupStatesMode::Flat => group_indices, + GroupStatesMode::Blocked(blk_size) => { + self.group_idx_convert_buffer.clear(); + + let converted_group_indices = group_indices.iter().map(|group_idx| { + let blk_id = + ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; + let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; + blk_id * blk_size + blk_offset + }); + self.group_idx_convert_buffer + .extend(converted_group_indices); + + &self.group_idx_convert_buffer + } + }; + // update counts with partial counts self.counts.resize(total_num_groups, 0); self.null_state.accumulate( diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 0f87e62353cb..1acbd6927077 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -387,10 +387,6 @@ impl GroupsAccumulator for CountGroupsAccumulator { assert_eq!(values.len(), 1, "single argument to update_batch"); let values = &values[0]; - // Add one to each group's counter for each non null, non - // filtered value - self.counts.resize(total_num_groups, 0); - // Maybe we should convert the `group_indices` let group_indices = match self.mode { GroupStatesMode::Flat => group_indices, @@ -410,6 +406,10 @@ impl GroupsAccumulator for CountGroupsAccumulator { } }; + // Add one to each group's counter for each non null, non + // filtered value + self.counts.resize(total_num_groups, 0); + accumulate_indices( group_indices, values.logical_nulls().as_ref(), @@ -437,8 +437,28 @@ impl GroupsAccumulator for CountGroupsAccumulator { assert_eq!(partial_counts.null_count(), 0); let partial_counts = partial_counts.values(); + // Maybe we should convert the `group_indices` + let group_indices = match self.mode { + GroupStatesMode::Flat => group_indices, + GroupStatesMode::Blocked(blk_size) => { + self.group_idx_convert_buffer.clear(); + + let converted_group_indices = group_indices.iter().map(|group_idx| { + let blk_id = + ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; + let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; + blk_id * blk_size + blk_offset + }); + self.group_idx_convert_buffer + .extend(converted_group_indices); + + &self.group_idx_convert_buffer + } + }; + // Adds the counts with the partial counts self.counts.resize(total_num_groups, 0); + match opt_filter { Some(filter) => filter .iter() diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 208761f6e35f..f8dc7c01ba12 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -60,7 +60,7 @@ pub trait GroupValues: Send { fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { return Err(DataFusionError::NotImplemented( - "only flat mode is not supported yet".to_string(), + "only flat mode group values is supported yet".to_string(), )); } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 79b7b79792e6..eb6dc266fc5e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -607,7 +607,7 @@ fn maybe_enable_blocked_group_states( let group_supports_blocked = group_values.supports_blocked_mode(); let accumulators_support_blocked = - accumulators.iter().any(|acc| acc.supports_blocked_mode()); + accumulators.iter().all(|acc| acc.supports_blocked_mode()); match (group_supports_blocked, accumulators_support_blocked) { (true, true) => { @@ -767,10 +767,12 @@ impl Stream for GroupedHashAggregateStream { } ExecutionState::ProducingBlocks(blocks) => { - if let Some(blk) = blocks { + let emit_to = if let Some(blk) = blocks { if *blk > 0 { self.exec_state = ExecutionState::ProducingBlocks(Some(*blk - 1)); + + EmitTo::CurrentBlock(true) } else { self.exec_state = if self.input_done { ExecutionState::Done @@ -781,9 +783,11 @@ impl Stream for GroupedHashAggregateStream { }; continue; } - } + } else { + EmitTo::CurrentBlock(false) + }; - let emit_result = self.emit(EmitTo::CurrentBlock(true), false); + let emit_result = self.emit(emit_to, false); if emit_result.is_err() { return Poll::Ready(Some(emit_result)); } From ed7c1b7ab00118babe9e0add5d31165d6db72f7f Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 16 Aug 2024 12:24:44 +0800 Subject: [PATCH 009/107] support blocked mode in NullState. --- .../groups_accumulator/accumulate.rs | 301 +++++++++++++++--- 1 file changed, 256 insertions(+), 45 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 ac23710ecd3b..cba8b5397a0a 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,12 +19,14 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator +use std::collections::VecDeque; + use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; use datafusion_common::Result; -use datafusion_expr_common::groups_accumulator::EmitTo; +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupStatesMode}; /// 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,26 +62,32 @@ 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_blocks: VecDeque, + + mode: GroupStatesMode, } impl Default for NullState { fn default() -> Self { - Self::new() + Self::new(GroupStatesMode::Flat) } } impl NullState { - pub fn new() -> Self { + pub fn new(mode: GroupStatesMode) -> Self { Self { - seen_values: BooleanBufferBuilder::new(0), + seen_values_blocks: VecDeque::new(), + mode, } } /// 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_blocks + .iter() + .map(|blk| blk.capacity() / 8) + .sum::() } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -141,15 +149,32 @@ impl NullState { // 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); + initialize_builder( + &mut self.seen_values_blocks, + GroupStatesMode::Flat, + total_num_groups, + false, + ); + let seen_values_blocks = &mut self.seen_values_blocks; match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { let iter = group_indices.iter().zip(data.iter()); for (&group_index, &new_value) in iter { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) & 0x00000000ffffffff) + as usize; + let blk_offset = + ((group_index as u64) & 0x00000000ffffffff) as usize; + seen_values_blocks[blk_id].set_bit(blk_offset, true); + } + } value_fn(group_index, new_value); } } @@ -176,7 +201,22 @@ impl NullState { // valid bit was set, real value let is_valid = (mask & index_mask) != 0; if is_valid { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id] + .set_bit(blk_offset, true); + } + } value_fn(group_index, new_value); } index_mask <<= 1; @@ -193,7 +233,21 @@ impl NullState { .for_each(|(i, (&group_index, &new_value))| { let is_valid = remainder_bits & (1 << i) != 0; if is_valid { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id].set_bit(blk_offset, true); + } + } value_fn(group_index, new_value); } }); @@ -210,7 +264,21 @@ impl NullState { .zip(filter.iter()) .for_each(|((&group_index, &new_value), filter_value)| { if let Some(true) = filter_value { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id].set_bit(blk_offset, true); + } + } value_fn(group_index, new_value); } }) @@ -228,7 +296,22 @@ impl NullState { .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id] + .set_bit(blk_offset, true); + } + } value_fn(group_index, new_value) } } @@ -262,8 +345,13 @@ impl NullState { // 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); + initialize_builder( + &mut self.seen_values_blocks, + GroupStatesMode::Flat, + total_num_groups, + false, + ); + let seen_values_blocks = &mut self.seen_values_blocks; // These could be made more performant by iterating in chunks of 64 bits at a time match (values.null_count() > 0, opt_filter) { @@ -273,7 +361,20 @@ impl NullState { // 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); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = + ((group_index as u64) & 0x00000000ffffffff) as usize; + seen_values_blocks[blk_id].set_bit(blk_offset, true); + } + } value_fn(group_index, new_value) }, ) @@ -287,7 +388,21 @@ impl NullState { .zip(nulls.iter()) .for_each(|((&group_index, new_value), is_valid)| { if is_valid { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id].set_bit(blk_offset, true); + } + } value_fn(group_index, new_value); } }) @@ -302,7 +417,21 @@ impl NullState { .zip(filter.iter()) .for_each(|((&group_index, new_value), filter_value)| { if let Some(true) = filter_value { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id].set_bit(blk_offset, true); + } + } value_fn(group_index, new_value); } }) @@ -317,7 +446,22 @@ impl NullState { .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - seen_values.set_bit(group_index, true); + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blk_id = ((group_index as u64 >> 32) + & 0x00000000ffffffff) + as usize; + let blk_offset = ((group_index as u64) + & 0x00000000ffffffff) + as usize; + seen_values_blocks[blk_id] + .set_bit(blk_offset, true); + } + } value_fn(group_index, new_value) } } @@ -332,42 +476,51 @@ impl NullState { /// /// resets the internal state appropriately // TODO: add unit tests for the `EmitTo::CurrentBlock` branch - pub fn build( - &mut self, - emit_to: EmitTo, - block_size: Option, - ) -> Result { - let nulls: BooleanBuffer = self.seen_values.finish(); + pub fn build(&mut self, emit_to: EmitTo) -> Result { + if self.seen_values_blocks.is_empty() { + return Ok(NullBuffer::new(BooleanBufferBuilder::new(0).finish())); + } let nulls = match emit_to { - EmitTo::All => nulls, + EmitTo::All => match self.mode { + GroupStatesMode::Flat => { + self.seen_values_blocks.back_mut().unwrap().finish() + } + GroupStatesMode::Blocked(blk_size) => { + let total_num = (self.seen_values_blocks.len() - 1) * blk_size + + self.seen_values_blocks.back().unwrap().len(); + let mut total_buffer = BooleanBufferBuilder::new(total_num); + + for blk in self.seen_values_blocks.iter_mut() { + let nulls = blk.finish(); + for seen in nulls.iter() { + total_buffer.append(seen); + } + } + + total_buffer.finish() + } + } EmitTo::First(n) => { + let blk = self.seen_values_blocks.back_mut().unwrap(); // split off the first N values in seen_values // // TODO make this more efficient rather than two // copies and bitwise manipulation + let nulls = blk.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); + blk.append(seen); } first_n_null } EmitTo::CurrentBlock(_) => { - let block_size = block_size.unwrap(); - - // 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(block_size).collect(); - // reset the existing seen buffer - for seen in nulls.iter().skip(block_size) { - self.seen_values.append(seen); - } - first_n_null + let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); + cur_blk.finish() } }; + Ok(NullBuffer::new(nulls)) } } @@ -468,15 +621,73 @@ pub fn accumulate_indices( /// /// All new entries are initialized to `default_value` fn initialize_builder( - builder: &mut BooleanBufferBuilder, + builder_blocks: &mut VecDeque, + mode: GroupStatesMode, 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); +) { + match mode { + // It flat mode, we just a single builder, and grow it constantly. + GroupStatesMode::Flat => { + if builder_blocks.is_empty() { + builder_blocks.push_back(BooleanBufferBuilder::new(0)); + } + + let builder = builder_blocks.back_mut().unwrap(); + if builder.len() < total_num_groups { + let new_groups = total_num_groups - builder.len(); + builder.append_n(new_groups, default_value); + } + } + // It blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + GroupStatesMode::Blocked(blk_size) => { + let (mut cur_blk_idx, exist_slots) = if !builder_blocks.is_empty() { + let cur_blk_idx = builder_blocks.len() - 1; + let exist_slots = (builder_blocks.len() - 1) * blk_size + + builder_blocks.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + let exist_blks = builder_blocks.len(); + + // Ensure blks are enough. + let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; + builder_blocks.reserve(new_blks); + for _ in 0..new_blks { + builder_blocks.push_back(BooleanBufferBuilder::new(blk_size)); + } + + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. + let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + builder_blocks[cur_blk_idx].append_n(new_slots, default_value); + return; + } else { + builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + } + + // Expand blks + let expand_blks = new_slots / blk_size; + for _ in 0..expand_blks { + builder_blocks[cur_blk_idx].append_n(blk_size, default_value); + cur_blk_idx += 1; + } + + // Expand the last blk. + let last_expand_slots = new_slots % blk_size; + builder_blocks + .back_mut() + .unwrap() + .append_n(last_expand_slots, default_value); + } } - builder } #[cfg(test)] From 8d9d0c072ba1b6a08cc3d531454a176fd71e7b21 Mon Sep 17 00:00:00 2001 From: kamille Date: Sat, 17 Aug 2024 15:30:49 +0800 Subject: [PATCH 010/107] define the `blocked_accumulate`, so that we wont change the logic for falt accumulators. --- .../expr-common/src/groups_accumulator.rs | 17 + .../src/aggregate/groups_accumulator.rs | 1 + .../groups_accumulator/accumulate.rs | 297 +----- .../groups_accumulator/blocked_accumulate.rs | 947 ++++++++++++++++++ .../aggregate/groups_accumulator/bool_op.rs | 2 +- .../aggregate/groups_accumulator/prim_op.rs | 71 +- datafusion/functions-aggregate/src/average.rs | 8 +- 7 files changed, 1036 insertions(+), 307 deletions(-) create mode 100644 datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 952721d38898..1a6b5e3d2680 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -78,6 +78,23 @@ pub enum GroupStatesMode { Blocked(usize), } +pub struct BlockedGroupIndex { + pub block_id: usize, + pub block_offset: usize, +} + +impl BlockedGroupIndex { + pub fn new(group_index: usize) -> Self { + let block_id = ((group_index as u64 >> 32) & 0x00000000ffffffff) as usize; + let block_offset = ((group_index as u64) & 0x00000000ffffffff) as usize; + + Self { + block_id, + block_offset, + } + } +} + /// `GroupAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index b6218a50983c..2462506f755a 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -22,6 +22,7 @@ pub mod accumulate; pub mod bool_op; pub mod nulls; pub mod prim_op; +pub mod blocked_accumulate; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, 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 cba8b5397a0a..aded6aab6334 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,15 +19,11 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator -use std::collections::VecDeque; - use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_common::Result; -use datafusion_expr_common::groups_accumulator::{EmitTo, GroupStatesMode}; - +use datafusion_expr_common::groups_accumulator::EmitTo; /// 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. /// @@ -62,32 +58,26 @@ 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: VecDeque, - - mode: GroupStatesMode, + seen_values: BooleanBufferBuilder, } impl Default for NullState { fn default() -> Self { - Self::new(GroupStatesMode::Flat) + Self::new() } } impl NullState { - pub fn new(mode: GroupStatesMode) -> Self { + pub fn new() -> Self { Self { - seen_values_blocks: VecDeque::new(), - mode, + seen_values: BooleanBufferBuilder::new(0), } } /// 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_blocks - .iter() - .map(|blk| blk.capacity() / 8) - .sum::() + self.seen_values.capacity() / 8 } /// Invokes `value_fn(group_index, value)` for each non null, non @@ -149,32 +139,15 @@ impl NullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - initialize_builder( - &mut self.seen_values_blocks, - GroupStatesMode::Flat, - total_num_groups, - false, - ); - let seen_values_blocks = &mut self.seen_values_blocks; + let seen_values = + initialize_builder(&mut self.seen_values, total_num_groups, false); match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { let iter = group_indices.iter().zip(data.iter()); for (&group_index, &new_value) in iter { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) & 0x00000000ffffffff) - as usize; - let blk_offset = - ((group_index as u64) & 0x00000000ffffffff) as usize; - seen_values_blocks[blk_id].set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value); } } @@ -201,22 +174,7 @@ impl NullState { // valid bit was set, real value let is_valid = (mask & index_mask) != 0; if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id] - .set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value); } index_mask <<= 1; @@ -233,21 +191,7 @@ impl NullState { .for_each(|(i, (&group_index, &new_value))| { let is_valid = remainder_bits & (1 << i) != 0; if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id].set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value); } }); @@ -264,21 +208,7 @@ impl NullState { .zip(filter.iter()) .for_each(|((&group_index, &new_value), filter_value)| { if let Some(true) = filter_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id].set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value); } }) @@ -296,22 +226,7 @@ impl NullState { .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id] - .set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value) } } @@ -345,13 +260,8 @@ impl NullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - initialize_builder( - &mut self.seen_values_blocks, - GroupStatesMode::Flat, - total_num_groups, - false, - ); - let seen_values_blocks = &mut self.seen_values_blocks; + let seen_values = + initialize_builder(&mut self.seen_values, total_num_groups, false); // These could be made more performant by iterating in chunks of 64 bits at a time match (values.null_count() > 0, opt_filter) { @@ -361,20 +271,7 @@ impl NullState { // buffer is big enough (start everything at valid) group_indices.iter().zip(data.iter()).for_each( |(&group_index, new_value)| { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = - ((group_index as u64) & 0x00000000ffffffff) as usize; - seen_values_blocks[blk_id].set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value) }, ) @@ -388,21 +285,7 @@ impl NullState { .zip(nulls.iter()) .for_each(|((&group_index, new_value), is_valid)| { if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id].set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value); } }) @@ -417,21 +300,7 @@ impl NullState { .zip(filter.iter()) .for_each(|((&group_index, new_value), filter_value)| { if let Some(true) = filter_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id].set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value); } }) @@ -446,22 +315,7 @@ impl NullState { .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blk_id = ((group_index as u64 >> 32) - & 0x00000000ffffffff) - as usize; - let blk_offset = ((group_index as u64) - & 0x00000000ffffffff) - as usize; - seen_values_blocks[blk_id] - .set_bit(blk_offset, true); - } - } + seen_values.set_bit(group_index, true); value_fn(group_index, new_value) } } @@ -475,53 +329,26 @@ impl NullState { /// for the `emit_to` rows. /// /// resets the internal state appropriately - // TODO: add unit tests for the `EmitTo::CurrentBlock` branch - pub fn build(&mut self, emit_to: EmitTo) -> Result { - if self.seen_values_blocks.is_empty() { - return Ok(NullBuffer::new(BooleanBufferBuilder::new(0).finish())); - } + pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { + let nulls: BooleanBuffer = self.seen_values.finish(); let nulls = match emit_to { - EmitTo::All => match self.mode { - GroupStatesMode::Flat => { - self.seen_values_blocks.back_mut().unwrap().finish() - } - GroupStatesMode::Blocked(blk_size) => { - let total_num = (self.seen_values_blocks.len() - 1) * blk_size - + self.seen_values_blocks.back().unwrap().len(); - let mut total_buffer = BooleanBufferBuilder::new(total_num); - - for blk in self.seen_values_blocks.iter_mut() { - let nulls = blk.finish(); - for seen in nulls.iter() { - total_buffer.append(seen); - } - } - - total_buffer.finish() - } - } + EmitTo::All => nulls, EmitTo::First(n) => { - let blk = self.seen_values_blocks.back_mut().unwrap(); // split off the first N values in seen_values // // TODO make this more efficient rather than two // copies and bitwise manipulation - let nulls = blk.finish(); let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); // reset the existing seen buffer for seen in nulls.iter().skip(n) { - blk.append(seen); + self.seen_values.append(seen); } first_n_null } - EmitTo::CurrentBlock(_) => { - let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); - cur_blk.finish() - } + EmitTo::CurrentBlock(_) => unreachable!("can't support blocked emission in flat NullState"), }; - - Ok(NullBuffer::new(nulls)) + NullBuffer::new(nulls) } } @@ -621,73 +448,15 @@ pub fn accumulate_indices( /// /// All new entries are initialized to `default_value` fn initialize_builder( - builder_blocks: &mut VecDeque, - mode: GroupStatesMode, + builder: &mut BooleanBufferBuilder, total_num_groups: usize, default_value: bool, -) { - match mode { - // It flat mode, we just a single builder, and grow it constantly. - GroupStatesMode::Flat => { - if builder_blocks.is_empty() { - builder_blocks.push_back(BooleanBufferBuilder::new(0)); - } - - let builder = builder_blocks.back_mut().unwrap(); - if builder.len() < total_num_groups { - let new_groups = total_num_groups - builder.len(); - builder.append_n(new_groups, default_value); - } - } - // It blocked mode, we ensure the blks are enough first, - // and then ensure slots in blks are enough. - GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if !builder_blocks.is_empty() { - let cur_blk_idx = builder_blocks.len() - 1; - let exist_slots = (builder_blocks.len() - 1) * blk_size - + builder_blocks.back().unwrap().len(); - - (cur_blk_idx, exist_slots) - } else { - (0, 0) - }; - let exist_blks = builder_blocks.len(); - - // Ensure blks are enough. - let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - builder_blocks.reserve(new_blks); - for _ in 0..new_blks { - builder_blocks.push_back(BooleanBufferBuilder::new(blk_size)); - } - - // Ensure slots are enough. - let mut new_slots = total_num_groups - exist_slots; - // Expand current blk. - let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - builder_blocks[cur_blk_idx].append_n(new_slots, default_value); - return; - } else { - builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; - } - - // Expand blks - let expand_blks = new_slots / blk_size; - for _ in 0..expand_blks { - builder_blocks[cur_blk_idx].append_n(blk_size, default_value); - cur_blk_idx += 1; - } - - // Expand the last blk. - let last_expand_slots = new_slots % blk_size; - builder_blocks - .back_mut() - .unwrap() - .append_n(last_expand_slots, default_value); - } +) -> &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)] @@ -944,7 +713,7 @@ mod test { // 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, None).unwrap(); + let null_buffer = null_state.build(EmitTo::All); assert_eq!(null_buffer, expected_null_buffer); } @@ -1061,7 +830,7 @@ mod test { // 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, None).unwrap(); + let null_buffer = null_state.build(EmitTo::All); assert_eq!(null_buffer, expected_null_buffer); } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs new file mode 100644 index 000000000000..3adfff4019ba --- /dev/null +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs @@ -0,0 +1,947 @@ +// 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. + +//! [`GroupsAccumulator`] helpers: [`NullState`] and [`accumulate_indices`] +//! +//! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator + +use std::collections::VecDeque; + +use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; +use arrow::buffer::{BooleanBuffer, NullBuffer}; +use arrow::datatypes::ArrowPrimitiveType; + +use datafusion_common::Result; +use datafusion_expr_common::groups_accumulator::{ + BlockedGroupIndex, EmitTo, GroupStatesMode, +}; + +/// Similar as the [NullState] but supported the blocked version accumulator +#[derive(Debug)] +pub struct BlockedNullState { + /// 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` + seen_values_blocks: VecDeque, + + mode: GroupStatesMode, +} + +impl Default for BlockedNullState { + fn default() -> Self { + Self::new(GroupStatesMode::Flat) + } +} + +impl BlockedNullState { + pub fn new(mode: GroupStatesMode) -> Self { + Self { + seen_values_blocks: VecDeque::new(), + mode, + } + } + + /// 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_blocks + .iter() + .map(|blk| blk.capacity() / 8) + .sum::() + } + + /// Similar as [NullState::accumulate] but supported the blocked version accumulator + pub fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + mut value_fn: F, + ) where + T: ArrowPrimitiveType + Send, + F: FnMut(usize, T::Native) + Send, + { + let data: &[T::Native] = values.values(); + assert_eq!(data.len(), group_indices.len()); + + // ensure the seen_values is big enough (start everything at + // "not seen" valid) + ensure_enough_room_for_nulls( + &mut self.seen_values_blocks, + GroupStatesMode::Flat, + total_num_groups, + false, + ); + let seen_values_blocks = &mut self.seen_values_blocks; + + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + let iter = group_indices.iter().zip(data.iter()); + for (&group_index, &new_value) in iter { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value); + } + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let data_chunks = data.chunks_exact(64); + let bit_chunks = nulls.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + let data_remainder = data_chunks.remainder(); + + group_indices_chunks + .zip(data_chunks) + .zip(bit_chunks.iter()) + .for_each(|((group_index_chunk, data_chunk), mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().zip(data_chunk.iter()).for_each( + |(&group_index, &new_value)| { + // valid bit was set, real value + let is_valid = (mask & index_mask) != 0; + if is_valid { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit( + blocked_index.block_offset, + true, + ); + } + } + value_fn(group_index, new_value); + } + index_mask <<= 1; + }, + ) + }); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .zip(data_remainder.iter()) + .enumerate() + .for_each(|(i, (&group_index, &new_value))| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value); + } + }); + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, &new_value), filter_value)| { + if let Some(true) = filter_value { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value) + } + } + }) + } + } + } + + /// Similar as [NullState::build] but support the blocked version accumulator + pub fn build(&mut self, emit_to: EmitTo) -> Result { + if self.seen_values_blocks.is_empty() { + return Ok(NullBuffer::new(BooleanBufferBuilder::new(0).finish())); + } + + let nulls = match emit_to { + EmitTo::All => match self.mode { + GroupStatesMode::Flat => { + self.seen_values_blocks.back_mut().unwrap().finish() + } + GroupStatesMode::Blocked(blk_size) => { + let total_num = (self.seen_values_blocks.len() - 1) * blk_size + + self.seen_values_blocks.back().unwrap().len(); + let mut total_buffer = BooleanBufferBuilder::new(total_num); + + for blk in self.seen_values_blocks.iter_mut() { + let nulls = blk.finish(); + for seen in nulls.iter() { + total_buffer.append(seen); + } + } + + total_buffer.finish() + } + }, + EmitTo::First(n) => { + let blk = self.seen_values_blocks.back_mut().unwrap(); + // split off the first N values in seen_values + // + // TODO make this more efficient rather than two + // copies and bitwise manipulation + let nulls = blk.finish(); + let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); + // reset the existing seen buffer + for seen in nulls.iter().skip(n) { + blk.append(seen); + } + first_n_null + } + EmitTo::CurrentBlock(_) => { + let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); + cur_blk.finish() + } + }; + + Ok(NullBuffer::new(nulls)) + } +} + +/// This function is called to update the accumulator state per row +/// when the value is not needed (e.g. COUNT) +/// +/// `F`: Invoked like `value_fn(group_index) for all non null values +/// passing the filter. Note that no tracking is done for null inputs +/// or which groups have seen any values +/// +/// See [`NullState::accumulate`], for more details on other +/// arguments. +pub fn accumulate_indices( + group_indices: &[usize], + nulls: Option<&NullBuffer>, + opt_filter: Option<&BooleanArray>, + mut index_fn: F, +) where + F: FnMut(usize) + Send, +{ + match (nulls, opt_filter) { + (None, None) => { + for &group_index in group_indices.iter() { + index_fn(group_index) + } + } + (None, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + let iter = group_indices.iter().zip(filter.iter()); + for (&group_index, filter_value) in iter { + if let Some(true) = filter_value { + index_fn(group_index) + } + } + } + (Some(valids), None) => { + assert_eq!(valids.len(), group_indices.len()); + // This is based on (ahem, COPY/PASTA) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let bit_chunks = valids.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + + group_indices_chunks.zip(bit_chunks.iter()).for_each( + |(group_index_chunk, mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().for_each(|&group_index| { + // valid bit was set, real vale + let is_valid = (mask & index_mask) != 0; + if is_valid { + index_fn(group_index); + } + index_mask <<= 1; + }) + }, + ); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .enumerate() + .for_each(|(i, &group_index)| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + index_fn(group_index) + } + }); + } + + (Some(valids), Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + assert_eq!(valids.len(), group_indices.len()); + // The performance with a filter could likely be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(valids.iter()) + .for_each(|((filter_value, &group_index), is_valid)| { + if let (Some(true), true) = (filter_value, is_valid) { + index_fn(group_index) + } + }) + } + } +} + +/// Ensures that `builder` contains a `BooleanBufferBuilder with at +/// least `total_num_groups`. +/// +/// All new entries are initialized to `default_value` +fn ensure_enough_room_for_nulls( + builder_blocks: &mut VecDeque, + mode: GroupStatesMode, + total_num_groups: usize, + default_value: bool, +) { + match mode { + // It flat mode, we just a single builder, and grow it constantly. + GroupStatesMode::Flat => { + if builder_blocks.is_empty() { + builder_blocks.push_back(BooleanBufferBuilder::new(0)); + } + + let builder = builder_blocks.back_mut().unwrap(); + if builder.len() < total_num_groups { + let new_groups = total_num_groups - builder.len(); + builder.append_n(new_groups, default_value); + } + } + // It blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + GroupStatesMode::Blocked(blk_size) => { + let (mut cur_blk_idx, exist_slots) = if !builder_blocks.is_empty() { + let cur_blk_idx = builder_blocks.len() - 1; + let exist_slots = (builder_blocks.len() - 1) * blk_size + + builder_blocks.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + let exist_blks = builder_blocks.len(); + + // Ensure blks are enough. + let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; + builder_blocks.reserve(new_blks); + for _ in 0..new_blks { + builder_blocks.push_back(BooleanBufferBuilder::new(blk_size)); + } + + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. + let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + builder_blocks[cur_blk_idx].append_n(new_slots, default_value); + return; + } else { + builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + } + + // Expand blks + let expand_blks = new_slots / blk_size; + for _ in 0..expand_blks { + builder_blocks[cur_blk_idx].append_n(blk_size, default_value); + cur_blk_idx += 1; + } + + // Expand the last blk. + let last_expand_slots = new_slots % blk_size; + builder_blocks + .back_mut() + .unwrap() + .append_n(last_expand_slots, default_value); + } + } +} + +pub(crate) fn ensure_enough_room_for_values( + values: &mut VecDeque>, + mode: GroupStatesMode, + total_num_groups: usize, + default_value: T, +) { + match mode { + // It flat mode, we just a single builder, and grow it constantly. + GroupStatesMode::Flat => { + if values.is_empty() { + values.push_back(Vec::new()); + } + + let single = values.back_mut().unwrap(); + if single.len() < total_num_groups { + let new_groups = total_num_groups - single.len(); + single.resize(new_groups, default_value.clone()); + } + } + // It blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + GroupStatesMode::Blocked(blk_size) => { + let (mut cur_blk_idx, exist_slots) = if !values.is_empty() { + let cur_blk_idx = values.len() - 1; + let exist_slots = + (values.len() - 1) * blk_size + values.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + let exist_blks = values.len(); + + // Ensure blks are enough. + let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; + values.reserve(new_blks); + for _ in 0..new_blks { + values.push_back(Vec::with_capacity(blk_size)); + } + + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. + let cur_blk_rest_slots = blk_size - values[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + values[cur_blk_idx].resize(new_slots, default_value.clone()); + return; + } else { + values[cur_blk_idx].resize(cur_blk_rest_slots, default_value.clone()); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + } + + // Expand blks + let expand_blks = new_slots / blk_size; + for _ in 0..expand_blks { + values[cur_blk_idx].resize(blk_size, default_value.clone()); + cur_blk_idx += 1; + } + + // Expand the last blk. + let last_expand_slots = new_slots % blk_size; + values + .back_mut() + .unwrap() + .resize(last_expand_slots, default_value); + } + } +} + +#[cfg(test)] +mod test { + use super::*; + + use arrow::array::UInt32Array; + use rand::{rngs::ThreadRng, Rng}; + use std::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(); + + Fixture { + group_indices, + values, + values_with_nulls, + filter, + } + .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, + } + + 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(); + + // 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, + } + } + + /// 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); + + // nulls, no filters + Self::accumulate_test( + group_indices, + &values_with_nulls_array, + None, + total_num_groups, + ); + + // no nulls, filters + Self::accumulate_test( + group_indices, + &values_array, + Some(filter), + total_num_groups, + ); + + // nulls, filters + Self::accumulate_test( + group_indices, + &values_with_nulls_array, + Some(filter), + total_num_groups, + ); + } + + /// 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, + ) { + Self::accumulate_values_test( + group_indices, + values, + opt_filter, + total_num_groups, + ); + 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(); + Self::accumulate_boolean_test( + group_indices, + &boolean_values, + opt_filter, + total_num_groups, + ); + } + + /// 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, + ) { + let mut accumulated_values = vec![]; + let mut null_state = BlockedNullState::new(GroupStatesMode::Flat); + + null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_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.seen_values.finish_cloned(); + 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).unwrap(); + + 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, + ) { + let mut accumulated_values = vec![]; + let mut null_state = BlockedNullState::new(GroupStatesMode::Flat); + + null_state.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_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.seen_values.finish_cloned(); + 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).unwrap(); + + 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() + } + } +} 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 7ffc795ebe99..d205fc2dc901 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 @@ -124,7 +124,7 @@ where } }; - let nulls = self.null_state.build(emit_to, None)?; + let nulls = self.null_state.build(emit_to); let values = BooleanArray::new(values, Some(nulls)); Ok(Arc::new(values)) } 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 c8f891b5ea00..7d471929cb94 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,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::VecDeque; use std::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; @@ -24,9 +25,12 @@ use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{ - EmitTo, GroupStatesMode, GroupsAccumulator, + BlockedGroupIndex, EmitTo, GroupStatesMode, GroupsAccumulator, }; +use crate::aggregate::groups_accumulator::accumulate::ensure_enough_room_for_values; +use crate::aggregate::groups_accumulator::blocked_accumulate::BlockedNullState; + use super::accumulate::NullState; /// An accumulator that implements a single operation over @@ -45,7 +49,7 @@ where F: Fn(&mut T::Native, T::Native) + Send + Sync, { /// values per group, stored as the native type - values: Vec, + values_blocks: VecDeque>, /// The output type (needed for Decimal precision and scale) data_type: DataType, @@ -54,14 +58,12 @@ where starting_value: T::Native, /// Track nulls in the input / filters - null_state: NullState, + null_state: BlockedNullState, /// Function that computes the primitive result prim_fn: F, mode: GroupStatesMode, - - group_idx_convert_buffer: Vec, } impl PrimitiveGroupsAccumulator @@ -71,13 +73,12 @@ where { pub fn new(data_type: &DataType, prim_fn: F) -> Self { Self { - values: vec![], + values_blocks: VecDeque::new(), data_type: data_type.clone(), - null_state: NullState::new(), + null_state: BlockedNullState::new(GroupStatesMode::Flat), starting_value: T::default_value(), prim_fn, mode: GroupStatesMode::Flat, - group_idx_convert_buffer: Vec::new(), } } @@ -103,27 +104,13 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); - // Maybe we should convert the `group_indices` - let group_indices = match self.mode { - GroupStatesMode::Flat => group_indices, - GroupStatesMode::Blocked(blk_size) => { - self.group_idx_convert_buffer.clear(); - - let converted_group_indices = group_indices.iter().map(|group_idx| { - let blk_id = - ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; - let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; - blk_id * blk_size + blk_offset - }); - self.group_idx_convert_buffer - .extend(converted_group_indices); - - &self.group_idx_convert_buffer - } - }; - - // update values - self.values.resize(total_num_groups, self.starting_value); + // Ensure enough room in values + ensure_enough_room_for_values( + &mut self.values_blocks, + self.mode, + total_num_groups, + self.starting_value, + ); // NullState dispatches / handles tracking nulls and groups that saw no values self.null_state.accumulate( @@ -132,7 +119,19 @@ where opt_filter, total_num_groups, |group_index, new_value| { - let value = &mut self.values[group_index]; + let value = match self.mode { + GroupStatesMode::Flat => self + .values_blocks + .back_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.values_blocks[blocked_index.block_id] + [blocked_index.block_offset] + } + }; (self.prim_fn)(value, new_value); }, ); @@ -146,8 +145,8 @@ where GroupStatesMode::Blocked(blk_size) => Some(blk_size), }; - let values = emit_to.take_needed(&mut self.values, block_size); - let nulls = self.null_state.build(emit_to, block_size)?; + let values = emit_to.take_needed(&mut self.values_blocks, block_size); + 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()); @@ -228,7 +227,8 @@ where } fn size(&self) -> usize { - self.values.capacity() * std::mem::size_of::() + self.null_state.size() + self.values_blocks.capacity() * std::mem::size_of::() + + self.null_state.size() } fn supports_blocked_mode(&self) -> bool { @@ -236,9 +236,8 @@ where } fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { - self.values.clear(); - self.null_state = NullState::new(); - + self.values_blocks.clear(); + self.null_state = NullState::new(mode); self.mode = mode; Ok(()) diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 7797c051e789..2c8a80237680 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -409,8 +409,6 @@ where avg_fn: F, mode: GroupStatesMode, - - group_idx_convert_buffer: Vec, } impl AvgGroupsAccumulator @@ -429,10 +427,9 @@ where sum_data_type: sum_data_type.clone(), counts: vec![], sums: vec![], - null_state: NullState::new(), + null_state: NullState::new(GroupStatesMode::Flat), avg_fn, mode: GroupStatesMode::Flat, - group_idx_convert_buffer: Vec::new(), } } } @@ -650,8 +647,7 @@ where fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { self.counts.clear(); self.sums.clear(); - self.null_state = NullState::new(); - + self.null_state = NullState::new(mode); self.mode = mode; Ok(()) From cb53724d245aaf045792482ede8d32354faf56c7 Mon Sep 17 00:00:00 2001 From: kamille Date: Sat, 17 Aug 2024 16:33:24 +0800 Subject: [PATCH 011/107] impl blocked version prim_op and avg accumulators. --- .../expr-common/src/groups_accumulator.rs | 63 +++++++- .../src/aggregate/groups_accumulator.rs | 85 ++++++++++- .../groups_accumulator/blocked_accumulate.rs | 135 ------------------ .../aggregate/groups_accumulator/prim_op.rs | 12 +- datafusion/functions-aggregate/src/average.rs | 125 +++++++--------- 5 files changed, 202 insertions(+), 218 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 1a6b5e3d2680..1acc53b30afe 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,7 +17,7 @@ //! Vectorized [`GroupsAccumulator`] -use std::cmp::min; +use std::{cmp::min, collections::VecDeque, mem}; use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -70,6 +70,67 @@ impl EmitTo { } } } + + /// Removes the number of rows from `blocks` required to emit, + /// returning a `Vec` with elements taken. + /// + /// The detailed behavior in different emissions: + /// - For Emit::All, the groups in blocks will be merged into + /// single block and returned. + /// + /// - For Emit::First, it will be only supported in the flat, + /// similar as `take_needed`. + /// + /// - For Emit::CurrentBlock, the first block will be taken and return. + /// + pub fn take_needed_from_blocks( + &self, + blocks: &mut VecDeque>, + mode: GroupStatesMode, + ) -> Vec { + if blocks.is_empty() { + return Vec::new(); + } + + match self { + Self::All => { + match mode { + GroupStatesMode::Flat => { + blocks.pop_front().unwrap() + } + GroupStatesMode::Blocked(_) => { + let blocks = mem::take(blocks); + blocks + .into_iter() + .flat_map(|blk| blk.into_iter()) + .collect::>() + } + } + } + Self::First(n) => { + match mode { + GroupStatesMode::Flat => { + let block = blocks.back_mut().unwrap(); + let split_at = min(block.len(), *n); + + // get end n+1,.. values into t + let mut t = block.split_off(split_at); + // leave n+1,.. in v + std::mem::swap(block, &mut t); + t + } + GroupStatesMode::Blocked(_) => { + unreachable!( + "can't support Emit::First in blocked mode accumulator" + ); + } + } + } + EmitTo::CurrentBlock(_) => { + blocks.pop_front().unwrap() + } + } + } } #[derive(Debug, Clone, Copy)] diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 2462506f755a..19430cddc259 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -24,6 +24,8 @@ pub mod nulls; pub mod prim_op; pub mod blocked_accumulate; +use std::collections::VecDeque; + use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, compute, @@ -33,7 +35,7 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; -use datafusion_expr_common::accumulator::Accumulator; +use datafusion_expr_common::{accumulator::Accumulator, groups_accumulator::GroupStatesMode}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] @@ -408,3 +410,84 @@ pub(crate) fn slice_and_maybe_filter( Ok(sliced_arrays) } } + +/// Expend blocked values to a big enough size for holding `total_num_groups` groups. +/// +/// For example, +/// +/// before expanding: +/// values: [x, x, x], [x, x, x] (blocks=2, block_size=3) +/// total_num_groups: 8 +/// +/// After expanding: +/// values: [x, x, x], [x, x, x], [default, default, default] +/// +pub fn ensure_enough_room_for_values( + values: &mut VecDeque>, + mode: GroupStatesMode, + total_num_groups: usize, + default_value: T, +) { + match mode { + // It flat mode, we just a single builder, and grow it constantly. + GroupStatesMode::Flat => { + if values.is_empty() { + values.push_back(Vec::new()); + } + + let single = values.back_mut().unwrap(); + if single.len() < total_num_groups { + let new_groups = total_num_groups - single.len(); + single.resize(new_groups, default_value.clone()); + } + } + // It blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + GroupStatesMode::Blocked(blk_size) => { + let (mut cur_blk_idx, exist_slots) = if !values.is_empty() { + let cur_blk_idx = values.len() - 1; + let exist_slots = + (values.len() - 1) * blk_size + values.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + let exist_blks = values.len(); + + // Ensure blks are enough. + let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; + values.reserve(new_blks); + for _ in 0..new_blks { + values.push_back(Vec::with_capacity(blk_size)); + } + + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. + let cur_blk_rest_slots = blk_size - values[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + values[cur_blk_idx].resize(new_slots, default_value.clone()); + return; + } else { + values[cur_blk_idx].resize(cur_blk_rest_slots, default_value.clone()); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + } + + // Expand blks + let expand_blks = new_slots / blk_size; + for _ in 0..expand_blks { + values[cur_blk_idx].resize(blk_size, default_value.clone()); + cur_blk_idx += 1; + } + + // Expand the last blk. + let last_expand_slots = new_slots % blk_size; + values + .back_mut() + .unwrap() + .resize(last_expand_slots, default_value); + } + } +} diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs index 3adfff4019ba..c4b79a39f7b3 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs @@ -458,76 +458,6 @@ fn ensure_enough_room_for_nulls( } } -pub(crate) fn ensure_enough_room_for_values( - values: &mut VecDeque>, - mode: GroupStatesMode, - total_num_groups: usize, - default_value: T, -) { - match mode { - // It flat mode, we just a single builder, and grow it constantly. - GroupStatesMode::Flat => { - if values.is_empty() { - values.push_back(Vec::new()); - } - - let single = values.back_mut().unwrap(); - if single.len() < total_num_groups { - let new_groups = total_num_groups - single.len(); - single.resize(new_groups, default_value.clone()); - } - } - // It blocked mode, we ensure the blks are enough first, - // and then ensure slots in blks are enough. - GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if !values.is_empty() { - let cur_blk_idx = values.len() - 1; - let exist_slots = - (values.len() - 1) * blk_size + values.back().unwrap().len(); - - (cur_blk_idx, exist_slots) - } else { - (0, 0) - }; - let exist_blks = values.len(); - - // Ensure blks are enough. - let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - values.reserve(new_blks); - for _ in 0..new_blks { - values.push_back(Vec::with_capacity(blk_size)); - } - - // Ensure slots are enough. - let mut new_slots = total_num_groups - exist_slots; - // Expand current blk. - let cur_blk_rest_slots = blk_size - values[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - values[cur_blk_idx].resize(new_slots, default_value.clone()); - return; - } else { - values[cur_blk_idx].resize(cur_blk_rest_slots, default_value.clone()); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; - } - - // Expand blks - let expand_blks = new_slots / blk_size; - for _ in 0..expand_blks { - values[cur_blk_idx].resize(blk_size, default_value.clone()); - cur_blk_idx += 1; - } - - // Expand the last blk. - let last_expand_slots = new_slots % blk_size; - values - .back_mut() - .unwrap() - .resize(last_expand_slots, default_value); - } - } -} - #[cfg(test)] mod test { use super::*; @@ -838,71 +768,6 @@ mod test { 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, - ) { - let mut accumulated_values = vec![]; - let mut null_state = BlockedNullState::new(GroupStatesMode::Flat); - - null_state.accumulate_boolean( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_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.seen_values.finish_cloned(); - 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).unwrap(); - - assert_eq!(null_buffer, expected_null_buffer); - } } /// Parallel implementation of NullState to check expected values 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 7d471929cb94..ff8ceea1947d 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 @@ -28,10 +28,9 @@ use datafusion_expr_common::groups_accumulator::{ BlockedGroupIndex, EmitTo, GroupStatesMode, GroupsAccumulator, }; -use crate::aggregate::groups_accumulator::accumulate::ensure_enough_room_for_values; use crate::aggregate::groups_accumulator::blocked_accumulate::BlockedNullState; +use crate::aggregate::groups_accumulator::ensure_enough_room_for_values; -use super::accumulate::NullState; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -140,12 +139,7 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let block_size = match self.mode { - GroupStatesMode::Flat => None, - GroupStatesMode::Blocked(blk_size) => Some(blk_size), - }; - - let values = emit_to.take_needed(&mut self.values_blocks, block_size); + let values = emit_to.take_needed_from_blocks(&mut self.values_blocks, self.mode); 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()); @@ -237,7 +231,7 @@ where fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { self.values_blocks.clear(); - self.null_state = NullState::new(mode); + self.null_state = BlockedNullState::new(mode); self.mode = mode; Ok(()) diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 2c8a80237680..a0a3080f1776 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,7 +29,7 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::groups_accumulator::GroupStatesMode; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, GroupStatesMode}; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; @@ -37,7 +37,8 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::NullState; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::blocked_accumulate::BlockedNullState; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::{ filtered_null_mask, set_nulls, }; @@ -45,6 +46,7 @@ use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls: use datafusion_functions_aggregate_common::utils::DecimalAverager; use log::debug; use std::any::Any; +use std::collections::VecDeque; use std::fmt::Debug; use std::sync::Arc; @@ -397,13 +399,13 @@ where return_data_type: DataType, /// Count per group (use u64 to make UInt64Array) - counts: Vec, + counts: VecDeque>, /// Sums per group, stored as the native type - sums: Vec, + sums: VecDeque>, /// Track nulls in the input / filters - null_state: NullState, + null_state: BlockedNullState, /// Function that computes the final average (value / count) avg_fn: F, @@ -425,9 +427,9 @@ where Self { return_data_type: return_data_type.clone(), sum_data_type: sum_data_type.clone(), - counts: vec![], - sums: vec![], - null_state: NullState::new(GroupStatesMode::Flat), + counts: VecDeque::new(), + sums: VecDeque::new(), + null_state: BlockedNullState::new(GroupStatesMode::Flat), avg_fn, mode: GroupStatesMode::Flat, } @@ -449,28 +451,9 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); - // Maybe we should convert the `group_indices` - let group_indices = match self.mode { - GroupStatesMode::Flat => group_indices, - GroupStatesMode::Blocked(blk_size) => { - self.group_idx_convert_buffer.clear(); - - let converted_group_indices = group_indices.iter().map(|group_idx| { - let blk_id = - ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; - let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; - blk_id * blk_size + blk_offset - }); - self.group_idx_convert_buffer - .extend(converted_group_indices); - - &self.group_idx_convert_buffer - } - }; - // increment counts, update sums - self.counts.resize(total_num_groups, 0); - self.sums.resize(total_num_groups, T::default_value()); + ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); + ensure_enough_room_for_values(&mut self.sums, self.mode, total_num_groups, T::default_value()); self.null_state.accumulate( group_indices, @@ -478,10 +461,24 @@ where opt_filter, total_num_groups, |group_index, new_value| { - let sum = &mut self.sums[group_index]; - *sum = sum.add_wrapping(new_value); + let sum = match self.mode { + GroupStatesMode::Flat => self.sums.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.sums[blocked_index.block_id][blocked_index.block_offset] + } + }; + + let count = match self.mode { + GroupStatesMode::Flat => self.counts.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.counts[blocked_index.block_id][blocked_index.block_offset] + } + }; - self.counts[group_index] += 1; + *sum = sum.add_wrapping(new_value); + *count += 1; }, ); @@ -489,14 +486,9 @@ where } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let block_size = match self.mode { - GroupStatesMode::Flat => None, - GroupStatesMode::Blocked(blk_size) => Some(blk_size), - }; - - let counts = emit_to.take_needed(&mut self.counts, block_size); - let sums = emit_to.take_needed(&mut self.sums, block_size); - let nulls = self.null_state.build(emit_to, block_size)?; + let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); + let sums = emit_to.take_needed_from_blocks(&mut self.sums, self.mode); + let nulls = self.null_state.build(emit_to)?; assert_eq!(nulls.len(), sums.len()); assert_eq!(counts.len(), sums.len()); @@ -531,18 +523,13 @@ where // return arrays for sums and counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let block_size = match self.mode { - GroupStatesMode::Flat => None, - GroupStatesMode::Blocked(blk_size) => Some(blk_size), - }; - - let nulls = self.null_state.build(emit_to, block_size)?; + let nulls = self.null_state.build(emit_to)?; let nulls = Some(nulls); - let counts = emit_to.take_needed(&mut self.counts, block_size); + let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); let counts = UInt64Array::new(counts.into(), nulls.clone()); // zero copy - let sums = emit_to.take_needed(&mut self.sums, block_size); + let sums = emit_to.take_needed_from_blocks(&mut self.sums, self.mode); let sums = PrimitiveArray::::new(sums.into(), nulls) // zero copy .with_data_type(self.sum_data_type.clone()); @@ -564,46 +551,40 @@ where let partial_counts = values[0].as_primitive::(); let partial_sums = values[1].as_primitive::(); - // Maybe we should convert the `group_indices` - let group_indices = match self.mode { - GroupStatesMode::Flat => group_indices, - GroupStatesMode::Blocked(blk_size) => { - self.group_idx_convert_buffer.clear(); - - let converted_group_indices = group_indices.iter().map(|group_idx| { - let blk_id = - ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; - let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; - blk_id * blk_size + blk_offset - }); - self.group_idx_convert_buffer - .extend(converted_group_indices); - - &self.group_idx_convert_buffer - } - }; - // update counts with partial counts - self.counts.resize(total_num_groups, 0); + ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); self.null_state.accumulate( group_indices, partial_counts, opt_filter, total_num_groups, |group_index, partial_count| { - self.counts[group_index] += partial_count; + let count = match self.mode { + GroupStatesMode::Flat => self.counts.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.counts[blocked_index.block_id][blocked_index.block_offset] + } + }; + *count += partial_count; }, ); // update sums - self.sums.resize(total_num_groups, T::default_value()); + ensure_enough_room_for_values(&mut self.sums, self.mode, total_num_groups, T::default_value()); self.null_state.accumulate( group_indices, partial_sums, opt_filter, total_num_groups, |group_index, new_value: ::Native| { - let sum = &mut self.sums[group_index]; + let sum = match self.mode { + GroupStatesMode::Flat => self.sums.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.sums[blocked_index.block_id][blocked_index.block_offset] + } + }; *sum = sum.add_wrapping(new_value); }, ); @@ -647,7 +628,7 @@ where fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { self.counts.clear(); self.sums.clear(); - self.null_state = NullState::new(mode); + self.null_state = BlockedNullState::new(mode); self.mode = mode; Ok(()) From cce049d89ff1f489c6c2ac35db8d8df243fc4095 Mon Sep 17 00:00:00 2001 From: kamille Date: Sat, 17 Aug 2024 17:41:30 +0800 Subject: [PATCH 012/107] fix streming tests. --- .../expr-common/src/groups_accumulator.rs | 27 +- .../src/aggregate/groups_accumulator.rs | 37 +- .../groups_accumulator/accumulate.rs | 4 +- .../groups_accumulator/blocked_accumulate.rs | 703 +++++++++--------- .../aggregate/groups_accumulator/prim_op.rs | 1 - datafusion/functions-aggregate/src/average.rs | 44 +- 6 files changed, 423 insertions(+), 393 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 1acc53b30afe..92f69bca5cd6 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -93,20 +93,16 @@ impl EmitTo { } match self { - Self::All => { - match mode { - GroupStatesMode::Flat => { - blocks.pop_front().unwrap() - } - GroupStatesMode::Blocked(_) => { - let blocks = mem::take(blocks); - blocks - .into_iter() - .flat_map(|blk| blk.into_iter()) - .collect::>() - } + Self::All => match mode { + GroupStatesMode::Flat => blocks.pop_front().unwrap(), + GroupStatesMode::Blocked(_) => { + let blocks = mem::take(blocks); + blocks + .into_iter() + .flat_map(|blk| blk.into_iter()) + .collect::>() } - } + }, Self::First(n) => { match mode { GroupStatesMode::Flat => { @@ -126,9 +122,7 @@ impl EmitTo { } } } - EmitTo::CurrentBlock(_) => { - blocks.pop_front().unwrap() - } + EmitTo::CurrentBlock(_) => blocks.pop_front().unwrap(), } } } @@ -139,6 +133,7 @@ pub enum GroupStatesMode { Blocked(usize), } +#[derive(Debug, Clone, Copy)] pub struct BlockedGroupIndex { pub block_id: usize, pub block_offset: usize, diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 19430cddc259..74d193e9c40b 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -19,12 +19,12 @@ //! Adapter that makes [`GroupsAccumulator`] out of [`Accumulator`] pub mod accumulate; +pub mod blocked_accumulate; pub mod bool_op; pub mod nulls; pub mod prim_op; -pub mod blocked_accumulate; -use std::collections::VecDeque; +use std::{collections::VecDeque, iter}; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, @@ -35,8 +35,10 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; -use datafusion_expr_common::{accumulator::Accumulator, groups_accumulator::GroupStatesMode}; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; +use datafusion_expr_common::{ + accumulator::Accumulator, groups_accumulator::GroupStatesMode, +}; /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] /// @@ -412,16 +414,16 @@ pub(crate) fn slice_and_maybe_filter( } /// Expend blocked values to a big enough size for holding `total_num_groups` groups. -/// +/// /// For example, -/// +/// /// before expanding: /// values: [x, x, x], [x, x, x] (blocks=2, block_size=3) /// total_num_groups: 8 -/// +/// /// After expanding: /// values: [x, x, x], [x, x, x], [default, default, default] -/// +/// pub fn ensure_enough_room_for_values( values: &mut VecDeque>, mode: GroupStatesMode, @@ -435,11 +437,10 @@ pub fn ensure_enough_room_for_values( values.push_back(Vec::new()); } - let single = values.back_mut().unwrap(); - if single.len() < total_num_groups { - let new_groups = total_num_groups - single.len(); - single.resize(new_groups, default_value.clone()); - } + values + .back_mut() + .unwrap() + .resize(total_num_groups, default_value); } // It blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. @@ -464,13 +465,16 @@ pub fn ensure_enough_room_for_values( // Ensure slots are enough. let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. let cur_blk_rest_slots = blk_size - values[cur_blk_idx].len(); if cur_blk_rest_slots >= new_slots { - values[cur_blk_idx].resize(new_slots, default_value.clone()); + values[cur_blk_idx] + .extend(iter::repeat(default_value.clone()).take(new_slots)); return; } else { - values[cur_blk_idx].resize(cur_blk_rest_slots, default_value.clone()); + values[cur_blk_idx] + .extend(iter::repeat(default_value.clone()).take(cur_blk_rest_slots)); new_slots -= cur_blk_rest_slots; cur_blk_idx += 1; } @@ -478,7 +482,8 @@ pub fn ensure_enough_room_for_values( // Expand blks let expand_blks = new_slots / blk_size; for _ in 0..expand_blks { - values[cur_blk_idx].resize(blk_size, default_value.clone()); + values[cur_blk_idx] + .extend(iter::repeat(default_value.clone()).take(blk_size)); cur_blk_idx += 1; } @@ -487,7 +492,7 @@ pub fn ensure_enough_room_for_values( values .back_mut() .unwrap() - .resize(last_expand_slots, default_value); + .extend(iter::repeat(default_value.clone()).take(last_expand_slots)); } } } 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 aded6aab6334..79f0f5408f16 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -346,7 +346,9 @@ impl NullState { } first_n_null } - EmitTo::CurrentBlock(_) => unreachable!("can't support blocked emission in flat NullState"), + EmitTo::CurrentBlock(_) => { + unreachable!("can't support blocked emission in flat NullState") + } }; NullBuffer::new(nulls) } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs index c4b79a39f7b3..6ed9c0ef7b47 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs @@ -87,7 +87,7 @@ impl BlockedNullState { // "not seen" valid) ensure_enough_room_for_nulls( &mut self.seen_values_blocks, - GroupStatesMode::Flat, + self.mode, total_num_groups, false, ); @@ -105,6 +105,9 @@ impl BlockedNullState { .set_bit(group_index, true), GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); + // dbg!(seen_values_blocks.len()); + // dbg!(blocked_index.block_id); + // dbg!(blk_size); seen_values_blocks[blocked_index.block_id] .set_bit(blocked_index.block_offset, true); } @@ -460,353 +463,353 @@ fn ensure_enough_room_for_nulls( #[cfg(test)] mod test { - use super::*; - - use arrow::array::UInt32Array; - use rand::{rngs::ThreadRng, Rng}; - use std::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(); - - Fixture { - group_indices, - values, - values_with_nulls, - filter, - } - .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, - } - - 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(); - - // 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, - } - } - - /// 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); - - // nulls, no filters - Self::accumulate_test( - group_indices, - &values_with_nulls_array, - None, - total_num_groups, - ); - - // no nulls, filters - Self::accumulate_test( - group_indices, - &values_array, - Some(filter), - total_num_groups, - ); - - // nulls, filters - Self::accumulate_test( - group_indices, - &values_with_nulls_array, - Some(filter), - total_num_groups, - ); - } - - /// 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, - ) { - Self::accumulate_values_test( - group_indices, - values, - opt_filter, - total_num_groups, - ); - 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(); - Self::accumulate_boolean_test( - group_indices, - &boolean_values, - opt_filter, - total_num_groups, - ); - } - - /// 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, - ) { - let mut accumulated_values = vec![]; - let mut null_state = BlockedNullState::new(GroupStatesMode::Flat); - - null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_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.seen_values.finish_cloned(); - 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).unwrap(); - - 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:#?}"); - } - } - - /// 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() - } - } + // use super::*; + + // use arrow::array::UInt32Array; + // use rand::{rngs::ThreadRng, Rng}; + // use std::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(); + + // Fixture { + // group_indices, + // values, + // values_with_nulls, + // filter, + // } + // .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, + // } + + // 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(); + + // // 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, + // } + // } + + // /// 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); + + // // nulls, no filters + // Self::accumulate_test( + // group_indices, + // &values_with_nulls_array, + // None, + // total_num_groups, + // ); + + // // no nulls, filters + // Self::accumulate_test( + // group_indices, + // &values_array, + // Some(filter), + // total_num_groups, + // ); + + // // nulls, filters + // Self::accumulate_test( + // group_indices, + // &values_with_nulls_array, + // Some(filter), + // total_num_groups, + // ); + // } + + // /// 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, + // ) { + // Self::accumulate_values_test( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // ); + // 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(); + // Self::accumulate_boolean_test( + // group_indices, + // &boolean_values, + // opt_filter, + // total_num_groups, + // ); + // } + + // /// 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, + // ) { + // let mut accumulated_values = vec![]; + // let mut null_state = BlockedNullState::new(GroupStatesMode::Flat); + + // null_state.accumulate( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // |group_index, value| { + // accumulated_values.push((group_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.seen_values.finish_cloned(); + // 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).unwrap(); + + // 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:#?}"); + // } + // } + + // /// 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() + // } + // } } 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 ff8ceea1947d..333d8edcbd40 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 @@ -31,7 +31,6 @@ use datafusion_expr_common::groups_accumulator::{ use crate::aggregate::groups_accumulator::blocked_accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::ensure_enough_room_for_values; - /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as /// the input type (such as `Sum`) diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index a0a3080f1776..c05b340a202a 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -453,7 +453,12 @@ where // increment counts, update sums ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - ensure_enough_room_for_values(&mut self.sums, self.mode, total_num_groups, T::default_value()); + ensure_enough_room_for_values( + &mut self.sums, + self.mode, + total_num_groups, + T::default_value(), + ); self.null_state.accumulate( group_indices, @@ -462,18 +467,26 @@ where total_num_groups, |group_index, new_value| { let sum = match self.mode { - GroupStatesMode::Flat => self.sums.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Flat => { + self.sums.back_mut().unwrap().get_mut(group_index).unwrap() + } GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); &mut self.sums[blocked_index.block_id][blocked_index.block_offset] } }; - + let count = match self.mode { - GroupStatesMode::Flat => self.counts.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Flat => self + .counts + .back_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id][blocked_index.block_offset] + &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset] } }; @@ -560,10 +573,16 @@ where total_num_groups, |group_index, partial_count| { let count = match self.mode { - GroupStatesMode::Flat => self.counts.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Flat => self + .counts + .back_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id][blocked_index.block_offset] + &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset] } }; *count += partial_count; @@ -571,7 +590,12 @@ where ); // update sums - ensure_enough_room_for_values(&mut self.sums, self.mode, total_num_groups, T::default_value()); + ensure_enough_room_for_values( + &mut self.sums, + self.mode, + total_num_groups, + T::default_value(), + ); self.null_state.accumulate( group_indices, partial_sums, @@ -579,7 +603,9 @@ where total_num_groups, |group_index, new_value: ::Native| { let sum = match self.mode { - GroupStatesMode::Flat => self.sums.back_mut().unwrap().get_mut(group_index).unwrap(), + GroupStatesMode::Flat => { + self.sums.back_mut().unwrap().get_mut(group_index).unwrap() + } GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); &mut self.sums[blocked_index.block_id][blocked_index.block_offset] From 89da481bd6f0747235a89af46fa2bdd351703534 Mon Sep 17 00:00:00 2001 From: kamille Date: Sat, 17 Aug 2024 17:50:49 +0800 Subject: [PATCH 013/107] impl blocked version count accumulator. --- .../expr-common/src/groups_accumulator.rs | 21 +--- .../src/aggregate/groups_accumulator.rs | 4 +- datafusion/functions-aggregate/src/count.rs | 103 +++++++++--------- 3 files changed, 59 insertions(+), 69 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 92f69bca5cd6..7ae6cfc45f25 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -43,31 +43,22 @@ impl EmitTo { /// remaining values in `v`. /// /// This avoids copying if Self::All - pub fn take_needed(&self, v: &mut Vec, block_size: Option) -> Vec { + pub fn take_needed(&self, v: &mut Vec) -> Vec { match self { Self::All => { // Take the entire vector, leave new (empty) vector std::mem::take(v) } Self::First(n) => { - let split_at = min(v.len(), *n); - - // get end n+1,.. values into t - let mut t = v.split_off(split_at); - // leave n+1,.. in v - std::mem::swap(v, &mut t); - t - } - EmitTo::CurrentBlock(_) => { - let block_size = block_size.unwrap(); - let split_at = min(v.len(), block_size); - // get end n+1,.. values into t - let mut t = v.split_off(split_at); + let mut t = v.split_off(*n); // leave n+1,.. in v std::mem::swap(v, &mut t); t } + Self::CurrentBlock(_) => unreachable!( + "can not support blocked emission in take_needed, you should use take_needed_from_blocks" + ), } } @@ -122,7 +113,7 @@ impl EmitTo { } } } - EmitTo::CurrentBlock(_) => blocks.pop_front().unwrap(), + Self::CurrentBlock(_) => blocks.pop_front().unwrap(), } } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 74d193e9c40b..7dd78b746e06 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -278,7 +278,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, None); + let states = emit_to.take_needed(&mut self.states); let results: Vec = states .into_iter() @@ -297,7 +297,7 @@ impl GroupsAccumulator for GroupsAccumulatorAdapter { 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, None); + 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/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 1acbd6927077..1d54a439b608 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,9 +16,10 @@ // under the License. use ahash::RandomState; -use datafusion_expr::groups_accumulator::GroupStatesMode; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, GroupStatesMode}; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; -use std::collections::HashSet; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; +use std::collections::{HashSet, VecDeque}; use std::ops::BitAnd; use std::{fmt::Debug, sync::Arc}; @@ -359,19 +360,16 @@ struct CountGroupsAccumulator { /// output type of count is `DataType::Int64`. Thus by using `i64` /// for the counts, the output [`Int64Array`] can be created /// without copy. - counts: Vec, + counts: VecDeque>, mode: GroupStatesMode, - - group_idx_convert_buffer: Vec, } impl CountGroupsAccumulator { pub fn new() -> Self { Self { - counts: vec![], + counts: VecDeque::new(), mode: GroupStatesMode::Flat, - group_idx_convert_buffer: Vec::new(), } } } @@ -387,35 +385,29 @@ impl GroupsAccumulator for CountGroupsAccumulator { assert_eq!(values.len(), 1, "single argument to update_batch"); let values = &values[0]; - // Maybe we should convert the `group_indices` - let group_indices = match self.mode { - GroupStatesMode::Flat => group_indices, - GroupStatesMode::Blocked(blk_size) => { - self.group_idx_convert_buffer.clear(); - - let converted_group_indices = group_indices.iter().map(|group_idx| { - let blk_id = - ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; - let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; - blk_id * blk_size + blk_offset - }); - self.group_idx_convert_buffer - .extend(converted_group_indices); - - &self.group_idx_convert_buffer - } - }; - // Add one to each group's counter for each non null, non // filtered value - self.counts.resize(total_num_groups, 0); + ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); accumulate_indices( group_indices, values.logical_nulls().as_ref(), opt_filter, |group_index| { - self.counts[group_index] += 1; + let count = match self.mode { + GroupStatesMode::Flat => self + .counts + .back_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset] + } + }; + *count += 1; }, ); @@ -437,27 +429,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { assert_eq!(partial_counts.null_count(), 0); let partial_counts = partial_counts.values(); - // Maybe we should convert the `group_indices` - let group_indices = match self.mode { - GroupStatesMode::Flat => group_indices, - GroupStatesMode::Blocked(blk_size) => { - self.group_idx_convert_buffer.clear(); - - let converted_group_indices = group_indices.iter().map(|group_idx| { - let blk_id = - ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; - let blk_offset = ((*group_idx as u64) & 0x00000000ffffffff) as usize; - blk_id * blk_size + blk_offset - }); - self.group_idx_convert_buffer - .extend(converted_group_indices); - - &self.group_idx_convert_buffer - } - }; - // Adds the counts with the partial counts - self.counts.resize(total_num_groups, 0); + ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); match opt_filter { Some(filter) => filter @@ -466,12 +439,38 @@ impl GroupsAccumulator for CountGroupsAccumulator { .zip(partial_counts.iter()) .for_each(|((filter_value, &group_index), partial_count)| { if let Some(true) = filter_value { - self.counts[group_index] += partial_count; + let count = match self.mode { + GroupStatesMode::Flat => self + .counts + .back_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset] + } + }; + *count += partial_count; } }), None => group_indices.iter().zip(partial_counts.iter()).for_each( |(&group_index, partial_count)| { - self.counts[group_index] += partial_count; + let count = match self.mode { + GroupStatesMode::Flat => self + .counts + .back_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset] + } + }; + *count += partial_count; }, ), } @@ -484,7 +483,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { GroupStatesMode::Flat => None, GroupStatesMode::Blocked(blk_size) => Some(blk_size), }; - let counts = emit_to.take_needed(&mut self.counts, block_size); + let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); // Count is always non null (null inputs just don't contribute to the overall values) let nulls = None; @@ -499,7 +498,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { GroupStatesMode::Flat => None, GroupStatesMode::Blocked(blk_size) => Some(blk_size), }; - let counts = emit_to.take_needed(&mut self.counts, block_size); + let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls Ok(vec![Arc::new(counts) as ArrayRef]) From b2fc6d31fc977ccc98abb60d9b0bec230b918ee8 Mon Sep 17 00:00:00 2001 From: kamille Date: Sat, 17 Aug 2024 18:17:31 +0800 Subject: [PATCH 014/107] fix the special cast that total groups num is zero. --- datafusion/expr-common/src/groups_accumulator.rs | 4 ---- .../src/aggregate/groups_accumulator.rs | 4 ++++ .../aggregate/groups_accumulator/blocked_accumulate.rs | 4 ++++ datafusion/functions-aggregate/src/count.rs | 8 -------- .../physical-plan/src/aggregates/group_values/row.rs | 4 ---- 5 files changed, 8 insertions(+), 16 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 7ae6cfc45f25..46ab094024b0 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -235,10 +235,6 @@ pub trait GroupsAccumulator: Send { fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { - // if self.supports_blocked_mode() { - // panic!("kamille debug debug"); - // } - return Err(DataFusionError::NotImplemented( "only flat mode accumulator is supported yet".to_string(), )); diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 7dd78b746e06..a906226987e1 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -430,6 +430,10 @@ pub fn ensure_enough_room_for_values( total_num_groups: usize, default_value: T, ) { + if total_num_groups == 0 { + return; + } + match mode { // It flat mode, we just a single builder, and grow it constantly. GroupStatesMode::Flat => { diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs index 6ed9c0ef7b47..d26c7250da9b 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs @@ -397,6 +397,10 @@ fn ensure_enough_room_for_nulls( total_num_groups: usize, default_value: bool, ) { + if total_num_groups == 0 { + return; + } + match mode { // It flat mode, we just a single builder, and grow it constantly. GroupStatesMode::Flat => { diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 1d54a439b608..7dc468ce806f 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -479,10 +479,6 @@ impl GroupsAccumulator for CountGroupsAccumulator { } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let block_size = match self.mode { - GroupStatesMode::Flat => None, - GroupStatesMode::Blocked(blk_size) => Some(blk_size), - }; let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); // Count is always non null (null inputs just don't contribute to the overall values) @@ -494,10 +490,6 @@ impl GroupsAccumulator for CountGroupsAccumulator { // return arrays for counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let block_size = match self.mode { - GroupStatesMode::Flat => None, - GroupStatesMode::Blocked(blk_size) => Some(blk_size), - }; let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index e91da9d1504b..c244ad414ad6 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -277,10 +277,6 @@ impl GroupValues for GroupValuesRows { } }, EmitTo::First(n) => { - if matches!(self.mode, GroupStatesMode::Blocked(_)) { - panic!("kamille debug"); - } - let blk = group_values.back_mut().unwrap(); let groups_rows = blk.iter().take(n); let output = self.row_converter.convert_rows(groups_rows)?; From fc5d05ba3d2e447ab01a1a3c0a17ce863f72ea18 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 00:09:46 +0800 Subject: [PATCH 015/107] move `BlockedNullState` to accumulate.rs. --- .../src/aggregate/groups_accumulator.rs | 1 - .../groups_accumulator/accumulate.rs | 336 ++++++- .../groups_accumulator/blocked_accumulate.rs | 819 ------------------ .../aggregate/groups_accumulator/prim_op.rs | 4 +- datafusion/functions-aggregate/src/average.rs | 6 +- 5 files changed, 340 insertions(+), 826 deletions(-) delete mode 100644 datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index a906226987e1..a6d3f335b2bd 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -19,7 +19,6 @@ //! Adapter that makes [`GroupsAccumulator`] out of [`Accumulator`] pub mod accumulate; -pub mod blocked_accumulate; pub mod bool_op; pub mod nulls; pub mod prim_op; diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs index 79f0f5408f16..3979b6102c8c 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,15 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator +use std::collections::VecDeque; + use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_expr_common::groups_accumulator::EmitTo; +use datafusion_expr_common::groups_accumulator::{ + BlockedGroupIndex, EmitTo, GroupStatesMode, +}; /// 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. /// @@ -354,6 +358,261 @@ impl NullState { } } +/// Similar as the [NullState] but supported the blocked version accumulator +#[derive(Debug)] +pub struct BlockedNullState { + /// 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` + seen_values_blocks: VecDeque, + + mode: GroupStatesMode, +} + +impl Default for BlockedNullState { + fn default() -> Self { + Self::new(GroupStatesMode::Flat) + } +} + +impl BlockedNullState { + pub fn new(mode: GroupStatesMode) -> Self { + Self { + seen_values_blocks: VecDeque::new(), + mode, + } + } + + /// 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_blocks + .iter() + .map(|blk| blk.capacity() / 8) + .sum::() + } + + /// Similar as [NullState::accumulate] but supported the blocked version accumulator + pub fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + mut value_fn: F, + ) where + T: ArrowPrimitiveType + Send, + F: FnMut(usize, T::Native) + Send, + { + let data: &[T::Native] = values.values(); + assert_eq!(data.len(), group_indices.len()); + + // ensure the seen_values is big enough (start everything at + // "not seen" valid) + ensure_enough_room_for_nulls( + &mut self.seen_values_blocks, + self.mode, + total_num_groups, + false, + ); + let seen_values_blocks = &mut self.seen_values_blocks; + + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + let iter = group_indices.iter().zip(data.iter()); + for (&group_index, &new_value) in iter { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = BlockedGroupIndex::new(group_index); + // dbg!(seen_values_blocks.len()); + // dbg!(blocked_index.block_id); + // dbg!(blk_size); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value); + } + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let data_chunks = data.chunks_exact(64); + let bit_chunks = nulls.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + let data_remainder = data_chunks.remainder(); + + group_indices_chunks + .zip(data_chunks) + .zip(bit_chunks.iter()) + .for_each(|((group_index_chunk, data_chunk), mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().zip(data_chunk.iter()).for_each( + |(&group_index, &new_value)| { + // valid bit was set, real value + let is_valid = (mask & index_mask) != 0; + if is_valid { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit( + blocked_index.block_offset, + true, + ); + } + } + value_fn(group_index, new_value); + } + index_mask <<= 1; + }, + ) + }); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .zip(data_remainder.iter()) + .enumerate() + .for_each(|(i, (&group_index, &new_value))| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value); + } + }); + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, &new_value), filter_value)| { + if let Some(true) = filter_value { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + match self.mode { + GroupStatesMode::Flat => seen_values_blocks + .back_mut() + .unwrap() + .set_bit(group_index, true), + GroupStatesMode::Blocked(_) => { + let blocked_index = + BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + } + } + value_fn(group_index, new_value) + } + } + }) + } + } + } + + /// Similar as [NullState::build] but support the blocked version accumulator + pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { + if self.seen_values_blocks.is_empty() { + return NullBuffer::new(BooleanBufferBuilder::new(0).finish()); + } + + let nulls = match emit_to { + EmitTo::All => match self.mode { + GroupStatesMode::Flat => { + self.seen_values_blocks.back_mut().unwrap().finish() + } + GroupStatesMode::Blocked(blk_size) => { + let total_num = (self.seen_values_blocks.len() - 1) * blk_size + + self.seen_values_blocks.back().unwrap().len(); + let mut total_buffer = BooleanBufferBuilder::new(total_num); + + for blk in self.seen_values_blocks.iter_mut() { + let nulls = blk.finish(); + for seen in nulls.iter() { + total_buffer.append(seen); + } + } + + total_buffer.finish() + } + }, + EmitTo::CurrentBlock(_) => { + let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); + cur_blk.finish() + } + EmitTo::First(_) => { + unreachable!("can't support emitting first in in flat BlockedNullState") + } + }; + + NullBuffer::new(nulls) + } +} + /// This function is called to update the accumulator state per row /// when the value is not needed (e.g. COUNT) /// @@ -461,6 +720,81 @@ fn initialize_builder( builder } +/// Similar as the [initialize_builder] but supported the blocked version accumulator +fn ensure_enough_room_for_nulls( + builder_blocks: &mut VecDeque, + mode: GroupStatesMode, + total_num_groups: usize, + default_value: bool, +) { + if total_num_groups == 0 { + return; + } + + match mode { + // It flat mode, we just a single builder, and grow it constantly. + GroupStatesMode::Flat => { + if builder_blocks.is_empty() { + builder_blocks.push_back(BooleanBufferBuilder::new(0)); + } + + let builder = builder_blocks.back_mut().unwrap(); + if builder.len() < total_num_groups { + let new_groups = total_num_groups - builder.len(); + builder.append_n(new_groups, default_value); + } + } + // It blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + GroupStatesMode::Blocked(blk_size) => { + let (mut cur_blk_idx, exist_slots) = if !builder_blocks.is_empty() { + let cur_blk_idx = builder_blocks.len() - 1; + let exist_slots = (builder_blocks.len() - 1) * blk_size + + builder_blocks.back().unwrap().len(); + + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; + let exist_blks = builder_blocks.len(); + + // Ensure blks are enough. + let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; + builder_blocks.reserve(new_blks); + for _ in 0..new_blks { + builder_blocks.push_back(BooleanBufferBuilder::new(blk_size)); + } + + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. + let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + builder_blocks[cur_blk_idx].append_n(new_slots, default_value); + return; + } else { + builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + } + + // Expand blks + let expand_blks = new_slots / blk_size; + for _ in 0..expand_blks { + builder_blocks[cur_blk_idx].append_n(blk_size, default_value); + cur_blk_idx += 1; + } + + // Expand the last blk. + let last_expand_slots = new_slots % blk_size; + builder_blocks + .back_mut() + .unwrap() + .append_n(last_expand_slots, default_value); + } + } +} + #[cfg(test)] mod test { use super::*; diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs deleted file mode 100644 index d26c7250da9b..000000000000 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/blocked_accumulate.rs +++ /dev/null @@ -1,819 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! [`GroupsAccumulator`] helpers: [`NullState`] and [`accumulate_indices`] -//! -//! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator - -use std::collections::VecDeque; - -use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; -use arrow::buffer::{BooleanBuffer, NullBuffer}; -use arrow::datatypes::ArrowPrimitiveType; - -use datafusion_common::Result; -use datafusion_expr_common::groups_accumulator::{ - BlockedGroupIndex, EmitTo, GroupStatesMode, -}; - -/// Similar as the [NullState] but supported the blocked version accumulator -#[derive(Debug)] -pub struct BlockedNullState { - /// 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` - seen_values_blocks: VecDeque, - - mode: GroupStatesMode, -} - -impl Default for BlockedNullState { - fn default() -> Self { - Self::new(GroupStatesMode::Flat) - } -} - -impl BlockedNullState { - pub fn new(mode: GroupStatesMode) -> Self { - Self { - seen_values_blocks: VecDeque::new(), - mode, - } - } - - /// 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_blocks - .iter() - .map(|blk| blk.capacity() / 8) - .sum::() - } - - /// Similar as [NullState::accumulate] but supported the blocked version accumulator - pub fn accumulate( - &mut self, - group_indices: &[usize], - values: &PrimitiveArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - mut value_fn: F, - ) where - T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, - { - let data: &[T::Native] = values.values(); - assert_eq!(data.len(), group_indices.len()); - - // ensure the seen_values is big enough (start everything at - // "not seen" valid) - ensure_enough_room_for_nulls( - &mut self.seen_values_blocks, - self.mode, - total_num_groups, - false, - ); - let seen_values_blocks = &mut self.seen_values_blocks; - - match (values.null_count() > 0, opt_filter) { - // no nulls, no filter, - (false, None) => { - let iter = group_indices.iter().zip(data.iter()); - for (&group_index, &new_value) in iter { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - // dbg!(seen_values_blocks.len()); - // dbg!(blocked_index.block_id); - // dbg!(blk_size); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value); - } - } - // nulls, no filter - (true, None) => { - let nulls = values.nulls().unwrap(); - // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum - // iterate over in chunks of 64 bits for more efficient null checking - let group_indices_chunks = group_indices.chunks_exact(64); - let data_chunks = data.chunks_exact(64); - let bit_chunks = nulls.inner().bit_chunks(); - - let group_indices_remainder = group_indices_chunks.remainder(); - let data_remainder = data_chunks.remainder(); - - group_indices_chunks - .zip(data_chunks) - .zip(bit_chunks.iter()) - .for_each(|((group_index_chunk, data_chunk), mask)| { - // index_mask has value 1 << i in the loop - let mut index_mask = 1; - group_index_chunk.iter().zip(data_chunk.iter()).for_each( - |(&group_index, &new_value)| { - // valid bit was set, real value - let is_valid = (mask & index_mask) != 0; - if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit( - blocked_index.block_offset, - true, - ); - } - } - value_fn(group_index, new_value); - } - index_mask <<= 1; - }, - ) - }); - - // handle any remaining bits (after the initial 64) - let remainder_bits = bit_chunks.remainder_bits(); - group_indices_remainder - .iter() - .zip(data_remainder.iter()) - .enumerate() - .for_each(|(i, (&group_index, &new_value))| { - let is_valid = remainder_bits & (1 << i) != 0; - if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value); - } - }); - } - // no nulls, but a filter - (false, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than a single - // iterator. TODO file a ticket - group_indices - .iter() - .zip(data.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, &new_value), filter_value)| { - if let Some(true) = filter_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value); - } - }) - } - // both null values and filters - (true, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than using - // iterators. TODO file a ticket - filter - .iter() - .zip(group_indices.iter()) - .zip(values.iter()) - .for_each(|((filter_value, &group_index), new_value)| { - if let Some(true) = filter_value { - if let Some(new_value) = new_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .back_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value) - } - } - }) - } - } - } - - /// Similar as [NullState::build] but support the blocked version accumulator - pub fn build(&mut self, emit_to: EmitTo) -> Result { - if self.seen_values_blocks.is_empty() { - return Ok(NullBuffer::new(BooleanBufferBuilder::new(0).finish())); - } - - let nulls = match emit_to { - EmitTo::All => match self.mode { - GroupStatesMode::Flat => { - self.seen_values_blocks.back_mut().unwrap().finish() - } - GroupStatesMode::Blocked(blk_size) => { - let total_num = (self.seen_values_blocks.len() - 1) * blk_size - + self.seen_values_blocks.back().unwrap().len(); - let mut total_buffer = BooleanBufferBuilder::new(total_num); - - for blk in self.seen_values_blocks.iter_mut() { - let nulls = blk.finish(); - for seen in nulls.iter() { - total_buffer.append(seen); - } - } - - total_buffer.finish() - } - }, - EmitTo::First(n) => { - let blk = self.seen_values_blocks.back_mut().unwrap(); - // split off the first N values in seen_values - // - // TODO make this more efficient rather than two - // copies and bitwise manipulation - let nulls = blk.finish(); - let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); - // reset the existing seen buffer - for seen in nulls.iter().skip(n) { - blk.append(seen); - } - first_n_null - } - EmitTo::CurrentBlock(_) => { - let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); - cur_blk.finish() - } - }; - - Ok(NullBuffer::new(nulls)) - } -} - -/// This function is called to update the accumulator state per row -/// when the value is not needed (e.g. COUNT) -/// -/// `F`: Invoked like `value_fn(group_index) for all non null values -/// passing the filter. Note that no tracking is done for null inputs -/// or which groups have seen any values -/// -/// See [`NullState::accumulate`], for more details on other -/// arguments. -pub fn accumulate_indices( - group_indices: &[usize], - nulls: Option<&NullBuffer>, - opt_filter: Option<&BooleanArray>, - mut index_fn: F, -) where - F: FnMut(usize) + Send, -{ - match (nulls, opt_filter) { - (None, None) => { - for &group_index in group_indices.iter() { - index_fn(group_index) - } - } - (None, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than a single - // iterator. TODO file a ticket - let iter = group_indices.iter().zip(filter.iter()); - for (&group_index, filter_value) in iter { - if let Some(true) = filter_value { - index_fn(group_index) - } - } - } - (Some(valids), None) => { - assert_eq!(valids.len(), group_indices.len()); - // This is based on (ahem, COPY/PASTA) arrow::compute::aggregate::sum - // iterate over in chunks of 64 bits for more efficient null checking - let group_indices_chunks = group_indices.chunks_exact(64); - let bit_chunks = valids.inner().bit_chunks(); - - let group_indices_remainder = group_indices_chunks.remainder(); - - group_indices_chunks.zip(bit_chunks.iter()).for_each( - |(group_index_chunk, mask)| { - // index_mask has value 1 << i in the loop - let mut index_mask = 1; - group_index_chunk.iter().for_each(|&group_index| { - // valid bit was set, real vale - let is_valid = (mask & index_mask) != 0; - if is_valid { - index_fn(group_index); - } - index_mask <<= 1; - }) - }, - ); - - // handle any remaining bits (after the initial 64) - let remainder_bits = bit_chunks.remainder_bits(); - group_indices_remainder - .iter() - .enumerate() - .for_each(|(i, &group_index)| { - let is_valid = remainder_bits & (1 << i) != 0; - if is_valid { - index_fn(group_index) - } - }); - } - - (Some(valids), Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - assert_eq!(valids.len(), group_indices.len()); - // The performance with a filter could likely be improved by - // iterating over the filter in chunks, rather than using - // iterators. TODO file a ticket - filter - .iter() - .zip(group_indices.iter()) - .zip(valids.iter()) - .for_each(|((filter_value, &group_index), is_valid)| { - if let (Some(true), true) = (filter_value, is_valid) { - index_fn(group_index) - } - }) - } - } -} - -/// Ensures that `builder` contains a `BooleanBufferBuilder with at -/// least `total_num_groups`. -/// -/// All new entries are initialized to `default_value` -fn ensure_enough_room_for_nulls( - builder_blocks: &mut VecDeque, - mode: GroupStatesMode, - total_num_groups: usize, - default_value: bool, -) { - if total_num_groups == 0 { - return; - } - - match mode { - // It flat mode, we just a single builder, and grow it constantly. - GroupStatesMode::Flat => { - if builder_blocks.is_empty() { - builder_blocks.push_back(BooleanBufferBuilder::new(0)); - } - - let builder = builder_blocks.back_mut().unwrap(); - if builder.len() < total_num_groups { - let new_groups = total_num_groups - builder.len(); - builder.append_n(new_groups, default_value); - } - } - // It blocked mode, we ensure the blks are enough first, - // and then ensure slots in blks are enough. - GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if !builder_blocks.is_empty() { - let cur_blk_idx = builder_blocks.len() - 1; - let exist_slots = (builder_blocks.len() - 1) * blk_size - + builder_blocks.back().unwrap().len(); - - (cur_blk_idx, exist_slots) - } else { - (0, 0) - }; - let exist_blks = builder_blocks.len(); - - // Ensure blks are enough. - let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - builder_blocks.reserve(new_blks); - for _ in 0..new_blks { - builder_blocks.push_back(BooleanBufferBuilder::new(blk_size)); - } - - // Ensure slots are enough. - let mut new_slots = total_num_groups - exist_slots; - // Expand current blk. - let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - builder_blocks[cur_blk_idx].append_n(new_slots, default_value); - return; - } else { - builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; - } - - // Expand blks - let expand_blks = new_slots / blk_size; - for _ in 0..expand_blks { - builder_blocks[cur_blk_idx].append_n(blk_size, default_value); - cur_blk_idx += 1; - } - - // Expand the last blk. - let last_expand_slots = new_slots % blk_size; - builder_blocks - .back_mut() - .unwrap() - .append_n(last_expand_slots, default_value); - } - } -} - -#[cfg(test)] -mod test { - // use super::*; - - // use arrow::array::UInt32Array; - // use rand::{rngs::ThreadRng, Rng}; - // use std::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(); - - // Fixture { - // group_indices, - // values, - // values_with_nulls, - // filter, - // } - // .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, - // } - - // 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(); - - // // 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, - // } - // } - - // /// 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); - - // // nulls, no filters - // Self::accumulate_test( - // group_indices, - // &values_with_nulls_array, - // None, - // total_num_groups, - // ); - - // // no nulls, filters - // Self::accumulate_test( - // group_indices, - // &values_array, - // Some(filter), - // total_num_groups, - // ); - - // // nulls, filters - // Self::accumulate_test( - // group_indices, - // &values_with_nulls_array, - // Some(filter), - // total_num_groups, - // ); - // } - - // /// 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, - // ) { - // Self::accumulate_values_test( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // ); - // 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(); - // Self::accumulate_boolean_test( - // group_indices, - // &boolean_values, - // opt_filter, - // total_num_groups, - // ); - // } - - // /// 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, - // ) { - // let mut accumulated_values = vec![]; - // let mut null_state = BlockedNullState::new(GroupStatesMode::Flat); - - // null_state.accumulate( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // |group_index, value| { - // accumulated_values.push((group_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.seen_values.finish_cloned(); - // 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).unwrap(); - - // 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:#?}"); - // } - // } - - // /// 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() - // } - // } -} 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 333d8edcbd40..2a7324800a25 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 @@ -28,7 +28,7 @@ use datafusion_expr_common::groups_accumulator::{ BlockedGroupIndex, EmitTo, GroupStatesMode, GroupsAccumulator, }; -use crate::aggregate::groups_accumulator::blocked_accumulate::BlockedNullState; +use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::ensure_enough_room_for_values; /// An accumulator that implements a single operation over @@ -139,7 +139,7 @@ where fn evaluate(&mut self, emit_to: EmitTo) -> Result { let values = emit_to.take_needed_from_blocks(&mut self.values_blocks, self.mode); - let nulls = self.null_state.build(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()); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index c05b340a202a..9070a3f3e160 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ Accumulator, AggregateUDFImpl, EmitTo, GroupsAccumulator, ReversedUDAF, Signature, }; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::blocked_accumulate::BlockedNullState; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::BlockedNullState; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::{ filtered_null_mask, set_nulls, @@ -501,7 +501,7 @@ where fn evaluate(&mut self, emit_to: EmitTo) -> Result { let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); let sums = emit_to.take_needed_from_blocks(&mut self.sums, self.mode); - let nulls = self.null_state.build(emit_to)?; + let nulls = self.null_state.build(emit_to); assert_eq!(nulls.len(), sums.len()); assert_eq!(counts.len(), sums.len()); @@ -536,7 +536,7 @@ where // return arrays for sums and counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let nulls = self.null_state.build(emit_to)?; + let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); From 6092943ed55464889765d608083906c220f5cfeb Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 12:08:09 +0800 Subject: [PATCH 016/107] fix tests. --- .../groups_accumulator/accumulate.rs | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 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 3979b6102c8c..42500cdab08e 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -600,13 +600,26 @@ impl BlockedNullState { total_buffer.finish() } }, + EmitTo::First(n) => { + assert!(matches!(self.mode, GroupStatesMode::Flat)); + + let blk = self.seen_values_blocks.back_mut().unwrap(); + // split off the first N values in seen_values + // + // TODO make this more efficient rather than two + // copies and bitwise manipulation + let nulls = blk.finish(); + let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); + // reset the existing seen buffer + for seen in nulls.iter().skip(n) { + blk.append(seen); + } + first_n_null + } EmitTo::CurrentBlock(_) => { let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); cur_blk.finish() } - EmitTo::First(_) => { - unreachable!("can't support emitting first in in flat BlockedNullState") - } }; NullBuffer::new(nulls) From 6fcc831838d1b9e50e7da913a1c2e75dcc5bbec8 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 14:37:11 +0800 Subject: [PATCH 017/107] define the `Blocks` to replace `VecDeque`. --- .../src/aggregate/groups_accumulator.rs | 65 ++++++++++++++++++- 1 file changed, 64 insertions(+), 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index a6d3f335b2bd..049ff8e8ddc0 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -23,7 +23,7 @@ pub mod bool_op; pub mod nulls; pub mod prim_op; -use std::{collections::VecDeque, iter}; +use std::{collections::VecDeque, iter, mem}; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, @@ -499,3 +499,66 @@ pub fn ensure_enough_room_for_values( } } } + +pub enum Blocks { + Single(Vec), + Multiple(VecDeque>), +} + +impl Blocks { + fn new() -> Self { + Self::Single(Vec::new()) + } + + fn current(&self) -> Option<&Vec> { + match self { + Blocks::Single(blk) => Some(blk), + Blocks::Multiple(blks) => { + blks.back() + } + } + } + + fn current_mut(&mut self) -> Option<&mut Vec> { + match self { + Blocks::Single(blk) => Some(blk), + Blocks::Multiple(blks) => { + blks.back_mut() + } + } + } + + fn push_block(&mut self, block: Vec) { + loop { + match self { + // If found it is Single, convert to Multiple first + Blocks::Single(single) => { + let mut new_multiple = VecDeque::with_capacity(2); + let first_block = mem::take(single); + new_multiple.push_back(first_block); + + *self = Self::Multiple(new_multiple); + } + // If found it Multiple, just push the block into it + Blocks::Multiple(multiple) => { + multiple.push_back(block); + break; + } + } + } + } + + fn pop_current_block(&mut self) -> Option> { + match self { + Blocks::Single(single) => Some(mem::take(single)), + Blocks::Multiple(multiple) => multiple.pop_front(), + } + } + + fn is_empty(&self) -> bool { + match self { + Blocks::Single(single) => single.is_empty(), + Blocks::Multiple(multiple) => multiple.is_empty(), + } + } +} \ No newline at end of file From 00cfa06c4c2040357084b2b342d2301ac49c2796 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 15:55:04 +0800 Subject: [PATCH 018/107] introduce Block to accumulators. --- .../expr-common/src/groups_accumulator.rs | 152 ++++++++++++++++-- .../src/aggregate/groups_accumulator.rs | 125 ++++---------- .../aggregate/groups_accumulator/prim_op.rs | 9 +- datafusion/functions-aggregate/src/average.rs | 33 ++-- datafusion/functions-aggregate/src/count.rs | 14 +- 5 files changed, 206 insertions(+), 127 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 46ab094024b0..2d0448e216b8 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,10 +17,16 @@ //! Vectorized [`GroupsAccumulator`] -use std::{cmp::min, collections::VecDeque, mem}; +use std::{ + cmp::min, + collections::VecDeque, + mem, + ops::{Index, IndexMut}, +}; use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; +use std::fmt; /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy)] @@ -76,7 +82,7 @@ impl EmitTo { /// pub fn take_needed_from_blocks( &self, - blocks: &mut VecDeque>, + blocks: &mut Blocks, mode: GroupStatesMode, ) -> Vec { if blocks.is_empty() { @@ -85,19 +91,16 @@ impl EmitTo { match self { Self::All => match mode { - GroupStatesMode::Flat => blocks.pop_front().unwrap(), + GroupStatesMode::Flat => blocks.pop_first_block().unwrap(), GroupStatesMode::Blocked(_) => { let blocks = mem::take(blocks); - blocks - .into_iter() - .flat_map(|blk| blk.into_iter()) - .collect::>() + blocks.into_to_vec() } }, Self::First(n) => { match mode { GroupStatesMode::Flat => { - let block = blocks.back_mut().unwrap(); + let block = blocks.current_mut().unwrap(); let split_at = min(block.len(), *n); // get end n+1,.. values into t @@ -113,7 +116,7 @@ impl EmitTo { } } } - Self::CurrentBlock(_) => blocks.pop_front().unwrap(), + Self::CurrentBlock(_) => blocks.pop_first_block().unwrap(), } } } @@ -142,6 +145,137 @@ impl BlockedGroupIndex { } } +pub enum Blocks { + Single(Vec), + Multiple(VecDeque>), +} + +impl Blocks { + pub fn new() -> Self { + Self::Single(Vec::new()) + } + + pub fn current(&self) -> Option<&Vec> { + match self { + Blocks::Single(blk) => Some(blk), + Blocks::Multiple(blks) => blks.back(), + } + } + + pub fn current_mut(&mut self) -> Option<&mut Vec> { + match self { + Blocks::Single(blk) => Some(blk), + Blocks::Multiple(blks) => blks.back_mut(), + } + } + + pub fn push_block(&mut self, block: Vec) { + loop { + match self { + // If found it is Single, convert to Multiple first + Blocks::Single(single) => { + let mut new_multiple = VecDeque::with_capacity(2); + let first_block = mem::take(single); + new_multiple.push_back(first_block); + + *self = Self::Multiple(new_multiple); + } + // If found it Multiple, just push the block into it + Blocks::Multiple(multiple) => { + multiple.push_back(block); + break; + } + } + } + } + + pub fn pop_first_block(&mut self) -> Option> { + match self { + Blocks::Single(single) => Some(mem::take(single)), + Blocks::Multiple(multiple) => multiple.pop_front(), + } + } + + fn is_empty(&self) -> bool { + match self { + Blocks::Single(single) => single.is_empty(), + Blocks::Multiple(multiple) => multiple.is_empty(), + } + } + + pub fn num_blocks(&self) -> usize { + match self { + Blocks::Single(_) => 1, + Blocks::Multiple(multiple) => multiple.len(), + } + } + + pub fn into_to_vec(self) -> Vec { + match self { + Blocks::Single(single) => single, + Blocks::Multiple(multiple) => { + multiple.into_iter().flat_map(|v| v.into_iter()).collect() + } + } + } + + pub fn capacity(&self) -> usize { + match self { + Blocks::Single(single) => single.capacity(), + Blocks::Multiple(multiple) => { + multiple.iter().map(|blk| blk.capacity()).sum::() + } + } + } + + pub fn clear(&mut self) { + *self = Self::new(); + } +} + +impl fmt::Debug for Blocks { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + match self { + Self::Single(single) => f.debug_tuple("Single").field(single).finish(), + Self::Multiple(multiple) => { + f.debug_tuple("Multiple").field(multiple).finish() + } + } + } +} + +impl Index for Blocks { + type Output = Vec; + + fn index(&self, index: usize) -> &Vec { + match self { + Blocks::Single(single) => { + assert!(index == 0); + single + } + Blocks::Multiple(multiple) => &multiple[index], + } + } +} + +impl IndexMut for Blocks { + fn index_mut(&mut self, index: usize) -> &mut Vec { + match self { + Blocks::Single(single) => { + assert!(index == 0); + single + } + Blocks::Multiple(multiple) => &mut multiple[index], + } + } +} + +impl Default for Blocks { + fn default() -> Self { + Self::new() + } +} + /// `GroupAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 049ff8e8ddc0..4f730de3d440 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -23,7 +23,7 @@ pub mod bool_op; pub mod nulls; pub mod prim_op; -use std::{collections::VecDeque, iter, mem}; +use std::iter; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, @@ -34,7 +34,7 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; -use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; +use datafusion_expr_common::groups_accumulator::{Blocks, EmitTo, GroupsAccumulator}; use datafusion_expr_common::{ accumulator::Accumulator, groups_accumulator::GroupStatesMode, }; @@ -424,7 +424,7 @@ pub(crate) fn slice_and_maybe_filter( /// values: [x, x, x], [x, x, x], [default, default, default] /// pub fn ensure_enough_room_for_values( - values: &mut VecDeque>, + values: &mut Blocks, mode: GroupStatesMode, total_num_groups: usize, default_value: T, @@ -436,34 +436,34 @@ pub fn ensure_enough_room_for_values( match mode { // It flat mode, we just a single builder, and grow it constantly. GroupStatesMode::Flat => { - if values.is_empty() { - values.push_back(Vec::new()); - } - values - .back_mut() + .current_mut() .unwrap() .resize(total_num_groups, default_value); } // It blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if !values.is_empty() { - let cur_blk_idx = values.len() - 1; - let exist_slots = - (values.len() - 1) * blk_size + values.back().unwrap().len(); + let (mut cur_blk_idx, exist_slots) = { + let cur_blk_idx = values.num_blocks() - 1; + let exist_slots = (values.num_blocks() - 1) * blk_size + + values.current().unwrap().len(); (cur_blk_idx, exist_slots) - } else { - (0, 0) }; - let exist_blks = values.len(); + // No new groups, don't need to expand, just return. + if exist_slots >= total_num_groups { + return; + } + + let exist_blks = values.num_blocks(); // Ensure blks are enough. let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - values.reserve(new_blks); - for _ in 0..new_blks { - values.push_back(Vec::with_capacity(blk_size)); + if new_blks > 0 { + for _ in 0..new_blks { + values.push_block(Vec::with_capacity(blk_size)); + } } // Ensure slots are enough. @@ -472,17 +472,19 @@ pub fn ensure_enough_room_for_values( // Expand current blk. let cur_blk_rest_slots = blk_size - values[cur_blk_idx].len(); if cur_blk_rest_slots >= new_slots { + // We just need to expand current blocks. values[cur_blk_idx] .extend(iter::repeat(default_value.clone()).take(new_slots)); return; - } else { - values[cur_blk_idx] - .extend(iter::repeat(default_value.clone()).take(cur_blk_rest_slots)); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; } - // Expand blks + // Expand current blk to full, and expand next blks + values[cur_blk_idx] + .extend(iter::repeat(default_value.clone()).take(cur_blk_rest_slots)); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + + // Expand whole blks let expand_blks = new_slots / blk_size; for _ in 0..expand_blks { values[cur_blk_idx] @@ -490,75 +492,14 @@ pub fn ensure_enough_room_for_values( cur_blk_idx += 1; } - // Expand the last blk. + // Expand the last blk if needed let last_expand_slots = new_slots % blk_size; - values - .back_mut() - .unwrap() - .extend(iter::repeat(default_value.clone()).take(last_expand_slots)); - } - } -} - -pub enum Blocks { - Single(Vec), - Multiple(VecDeque>), -} - -impl Blocks { - fn new() -> Self { - Self::Single(Vec::new()) - } - - fn current(&self) -> Option<&Vec> { - match self { - Blocks::Single(blk) => Some(blk), - Blocks::Multiple(blks) => { - blks.back() - } - } - } - - fn current_mut(&mut self) -> Option<&mut Vec> { - match self { - Blocks::Single(blk) => Some(blk), - Blocks::Multiple(blks) => { - blks.back_mut() - } - } - } - - fn push_block(&mut self, block: Vec) { - loop { - match self { - // If found it is Single, convert to Multiple first - Blocks::Single(single) => { - let mut new_multiple = VecDeque::with_capacity(2); - let first_block = mem::take(single); - new_multiple.push_back(first_block); - - *self = Self::Multiple(new_multiple); - } - // If found it Multiple, just push the block into it - Blocks::Multiple(multiple) => { - multiple.push_back(block); - break; - } + if last_expand_slots > 0 { + values + .current_mut() + .unwrap() + .extend(iter::repeat(default_value.clone()).take(last_expand_slots)); } } } - - fn pop_current_block(&mut self) -> Option> { - match self { - Blocks::Single(single) => Some(mem::take(single)), - Blocks::Multiple(multiple) => multiple.pop_front(), - } - } - - fn is_empty(&self) -> bool { - match self { - Blocks::Single(single) => single.is_empty(), - Blocks::Multiple(multiple) => multiple.is_empty(), - } - } -} \ No newline at end of file +} 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 2a7324800a25..5b662e0634a5 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::sync::Arc; use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; @@ -29,7 +28,7 @@ use datafusion_expr_common::groups_accumulator::{ }; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; -use crate::aggregate::groups_accumulator::ensure_enough_room_for_values; +use crate::aggregate::groups_accumulator::{ensure_enough_room_for_values, Blocks}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -47,7 +46,7 @@ where F: Fn(&mut T::Native, T::Native) + Send + Sync, { /// values per group, stored as the native type - values_blocks: VecDeque>, + values_blocks: Blocks, /// The output type (needed for Decimal precision and scale) data_type: DataType, @@ -71,7 +70,7 @@ where { pub fn new(data_type: &DataType, prim_fn: F) -> Self { Self { - values_blocks: VecDeque::new(), + values_blocks: Blocks::new(), data_type: data_type.clone(), null_state: BlockedNullState::new(GroupStatesMode::Flat), starting_value: T::default_value(), @@ -120,7 +119,7 @@ where let value = match self.mode { GroupStatesMode::Flat => self .values_blocks - .back_mut() + .current_mut() .unwrap() .get_mut(group_index) .unwrap(), diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 9070a3f3e160..993608f15a62 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,7 +29,7 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, GroupStatesMode}; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode}; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; @@ -46,7 +46,6 @@ use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls: use datafusion_functions_aggregate_common::utils::DecimalAverager; use log::debug; use std::any::Any; -use std::collections::VecDeque; use std::fmt::Debug; use std::sync::Arc; @@ -399,10 +398,10 @@ where return_data_type: DataType, /// Count per group (use u64 to make UInt64Array) - counts: VecDeque>, + counts: Blocks, /// Sums per group, stored as the native type - sums: VecDeque>, + sums: Blocks, /// Track nulls in the input / filters null_state: BlockedNullState, @@ -427,8 +426,8 @@ where Self { return_data_type: return_data_type.clone(), sum_data_type: sum_data_type.clone(), - counts: VecDeque::new(), - sums: VecDeque::new(), + counts: Blocks::new(), + sums: Blocks::new(), null_state: BlockedNullState::new(GroupStatesMode::Flat), avg_fn, mode: GroupStatesMode::Flat, @@ -467,9 +466,12 @@ where total_num_groups, |group_index, new_value| { let sum = match self.mode { - GroupStatesMode::Flat => { - self.sums.back_mut().unwrap().get_mut(group_index).unwrap() - } + GroupStatesMode::Flat => self + .sums + .current_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); &mut self.sums[blocked_index.block_id][blocked_index.block_offset] @@ -479,7 +481,7 @@ where let count = match self.mode { GroupStatesMode::Flat => self .counts - .back_mut() + .current_mut() .unwrap() .get_mut(group_index) .unwrap(), @@ -575,7 +577,7 @@ where let count = match self.mode { GroupStatesMode::Flat => self .counts - .back_mut() + .current_mut() .unwrap() .get_mut(group_index) .unwrap(), @@ -603,9 +605,12 @@ where total_num_groups, |group_index, new_value: ::Native| { let sum = match self.mode { - GroupStatesMode::Flat => { - self.sums.back_mut().unwrap().get_mut(group_index).unwrap() - } + GroupStatesMode::Flat => self + .sums + .current_mut() + .unwrap() + .get_mut(group_index) + .unwrap(), GroupStatesMode::Blocked(_) => { let blocked_index = BlockedGroupIndex::new(group_index); &mut self.sums[blocked_index.block_id][blocked_index.block_offset] diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 7dc468ce806f..4a08d7b615e1 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,10 +16,10 @@ // under the License. use ahash::RandomState; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, GroupStatesMode}; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode}; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; -use std::collections::{HashSet, VecDeque}; +use std::collections::HashSet; use std::ops::BitAnd; use std::{fmt::Debug, sync::Arc}; @@ -360,7 +360,7 @@ struct CountGroupsAccumulator { /// output type of count is `DataType::Int64`. Thus by using `i64` /// for the counts, the output [`Int64Array`] can be created /// without copy. - counts: VecDeque>, + counts: Blocks, mode: GroupStatesMode, } @@ -368,7 +368,7 @@ struct CountGroupsAccumulator { impl CountGroupsAccumulator { pub fn new() -> Self { Self { - counts: VecDeque::new(), + counts: Blocks::new(), mode: GroupStatesMode::Flat, } } @@ -397,7 +397,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { let count = match self.mode { GroupStatesMode::Flat => self .counts - .back_mut() + .current_mut() .unwrap() .get_mut(group_index) .unwrap(), @@ -442,7 +442,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { let count = match self.mode { GroupStatesMode::Flat => self .counts - .back_mut() + .current_mut() .unwrap() .get_mut(group_index) .unwrap(), @@ -460,7 +460,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { let count = match self.mode { GroupStatesMode::Flat => self .counts - .back_mut() + .current_mut() .unwrap() .get_mut(group_index) .unwrap(), From 261b0f1f1dc2c804678d72690a94b8a6e318bac4 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 17:05:53 +0800 Subject: [PATCH 019/107] make `Blocks` more general. --- .../expr-common/src/groups_accumulator.rs | 108 ++++++++++-------- .../src/aggregate/groups_accumulator.rs | 4 +- .../aggregate/groups_accumulator/prim_op.rs | 4 +- datafusion/functions-aggregate/src/average.rs | 6 +- datafusion/functions-aggregate/src/count.rs | 4 +- 5 files changed, 67 insertions(+), 59 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 2d0448e216b8..81e44d01c120 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -18,10 +18,7 @@ //! Vectorized [`GroupsAccumulator`] use std::{ - cmp::min, - collections::VecDeque, - mem, - ops::{Index, IndexMut}, + cmp::min, collections::VecDeque, iter, mem, ops::{Index, IndexMut} }; use arrow::array::{ArrayRef, BooleanArray}; @@ -82,13 +79,9 @@ impl EmitTo { /// pub fn take_needed_from_blocks( &self, - blocks: &mut Blocks, + blocks: &mut VecBlocks, mode: GroupStatesMode, ) -> Vec { - if blocks.is_empty() { - return Vec::new(); - } - match self { Self::All => match mode { GroupStatesMode::Flat => blocks.pop_first_block().unwrap(), @@ -146,37 +139,42 @@ impl BlockedGroupIndex { } pub enum Blocks { - Single(Vec), - Multiple(VecDeque>), + Single(Option), + Multiple(VecDeque), } impl Blocks { pub fn new() -> Self { - Self::Single(Vec::new()) + Self::Single(None) } - pub fn current(&self) -> Option<&Vec> { + pub fn current(&self) -> Option<&T> { match self { - Blocks::Single(blk) => Some(blk), + Blocks::Single(blk) => blk.as_ref(), Blocks::Multiple(blks) => blks.back(), } } - pub fn current_mut(&mut self) -> Option<&mut Vec> { + pub fn current_mut(&mut self) -> Option<&mut T> { match self { - Blocks::Single(blk) => Some(blk), + Blocks::Single(blk) => blk.as_mut(), Blocks::Multiple(blks) => blks.back_mut(), } } - pub fn push_block(&mut self, block: Vec) { + pub fn push_block(&mut self, block: T) { loop { match self { // If found it is Single, convert to Multiple first - Blocks::Single(single) => { + Blocks::Single(single_opt) => { + let single_opt = mem::take(single_opt); + if single_opt.is_none() { + *self = Self::Single(Some(block)); + break; + } + let mut new_multiple = VecDeque::with_capacity(2); - let first_block = mem::take(single); - new_multiple.push_back(first_block); + new_multiple.push_back(single_opt.unwrap()); *self = Self::Multiple(new_multiple); } @@ -189,42 +187,26 @@ impl Blocks { } } - pub fn pop_first_block(&mut self) -> Option> { + pub fn pop_first_block(&mut self) -> Option { match self { - Blocks::Single(single) => Some(mem::take(single)), + Blocks::Single(single) => mem::take(single), Blocks::Multiple(multiple) => multiple.pop_front(), } } - fn is_empty(&self) -> bool { - match self { - Blocks::Single(single) => single.is_empty(), - Blocks::Multiple(multiple) => multiple.is_empty(), - } - } - pub fn num_blocks(&self) -> usize { match self { - Blocks::Single(_) => 1, + Blocks::Single(None) => 0, + Blocks::Single(Some(_)) => 1, Blocks::Multiple(multiple) => multiple.len(), } } - pub fn into_to_vec(self) -> Vec { + pub fn iter(&self) -> Box + '_> { match self { - Blocks::Single(single) => single, - Blocks::Multiple(multiple) => { - multiple.into_iter().flat_map(|v| v.into_iter()).collect() - } - } - } - - pub fn capacity(&self) -> usize { - match self { - Blocks::Single(single) => single.capacity(), - Blocks::Multiple(multiple) => { - multiple.iter().map(|blk| blk.capacity()).sum::() - } + Blocks::Single(None) => Box::new(iter::empty()), + Blocks::Single(Some(single)) => Box::new(iter::once(single)), + Blocks::Multiple(multiple) => Box::new(multiple.iter()), } } @@ -245,37 +227,63 @@ impl fmt::Debug for Blocks { } impl Index for Blocks { - type Output = Vec; + type Output = T; - fn index(&self, index: usize) -> &Vec { + fn index(&self, index: usize) -> &T { match self { - Blocks::Single(single) => { + Blocks::Single(Some(single)) => { assert!(index == 0); single } Blocks::Multiple(multiple) => &multiple[index], + Blocks::Single(None) => unreachable!("can't use index to access empty blocks"), } } } impl IndexMut for Blocks { - fn index_mut(&mut self, index: usize) -> &mut Vec { + fn index_mut(&mut self, index: usize) -> &mut T { match self { - Blocks::Single(single) => { + Blocks::Single(Some(single)) => { assert!(index == 0); single } Blocks::Multiple(multiple) => &mut multiple[index], + Blocks::Single(None) => unreachable!("can't use index to access empty blocks"), } } } -impl Default for Blocks { +impl Default for VecBlocks { fn default() -> Self { Self::new() } } +pub type VecBlocks = Blocks>; + +impl VecBlocks { + pub fn into_to_vec(self) -> Vec { + match self { + Blocks::Single(None) => Vec::new(), + Blocks::Single(Some(single)) => single, + Blocks::Multiple(multiple) => { + multiple.into_iter().flat_map(|v| v.into_iter()).collect() + } + } + } + + pub fn capacity(&self) -> usize { + match self { + Blocks::Single(None) => 0, + Blocks::Single(Some(single)) => single.capacity(), + Blocks::Multiple(multiple) => { + multiple.iter().map(|blk| blk.capacity()).sum::() + } + } + } +} + /// `GroupAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 4f730de3d440..c67028892ba6 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -34,7 +34,7 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; -use datafusion_expr_common::groups_accumulator::{Blocks, EmitTo, GroupsAccumulator}; +use datafusion_expr_common::groups_accumulator::{Blocks, EmitTo, GroupsAccumulator, VecBlocks}; use datafusion_expr_common::{ accumulator::Accumulator, groups_accumulator::GroupStatesMode, }; @@ -424,7 +424,7 @@ pub(crate) fn slice_and_maybe_filter( /// values: [x, x, x], [x, x, x], [default, default, default] /// pub fn ensure_enough_room_for_values( - values: &mut Blocks, + values: &mut VecBlocks, mode: GroupStatesMode, total_num_groups: usize, default_value: T, 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 5b662e0634a5..954f16c1c354 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 @@ -24,7 +24,7 @@ use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{ - BlockedGroupIndex, EmitTo, GroupStatesMode, GroupsAccumulator, + BlockedGroupIndex, EmitTo, GroupStatesMode, GroupsAccumulator, VecBlocks, }; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; @@ -46,7 +46,7 @@ where F: Fn(&mut T::Native, T::Native) + Send + Sync, { /// values per group, stored as the native type - values_blocks: Blocks, + values_blocks: VecBlocks, /// The output type (needed for Decimal precision and scale) data_type: DataType, diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 993608f15a62..afc836f56448 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,7 +29,7 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode}; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks}; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; @@ -398,10 +398,10 @@ where return_data_type: DataType, /// Count per group (use u64 to make UInt64Array) - counts: Blocks, + counts: VecBlocks, /// Sums per group, stored as the native type - sums: Blocks, + sums: VecBlocks, /// Track nulls in the input / filters null_state: BlockedNullState, diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 4a08d7b615e1..50e855025481 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,7 +16,7 @@ // under the License. use ahash::RandomState; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode}; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks}; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; use std::collections::HashSet; @@ -360,7 +360,7 @@ struct CountGroupsAccumulator { /// output type of count is `DataType::Int64`. Thus by using `i64` /// for the counts, the output [`Int64Array`] can be created /// without copy. - counts: Blocks, + counts: VecBlocks, mode: GroupStatesMode, } From 53950b95200610a2aef8b4c3d58aff033d31268a Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 20:09:30 +0800 Subject: [PATCH 020/107] use `Blocks` in `BlockedNullState`. --- .../expr-common/src/groups_accumulator.rs | 21 ++++- .../src/aggregate/groups_accumulator.rs | 14 +++- .../groups_accumulator/accumulate.rs | 82 ++++++++++--------- datafusion/functions-aggregate/src/average.rs | 4 +- datafusion/functions-aggregate/src/count.rs | 4 +- 5 files changed, 80 insertions(+), 45 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 81e44d01c120..5adc02b8af13 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -18,7 +18,10 @@ //! Vectorized [`GroupsAccumulator`] use std::{ - cmp::min, collections::VecDeque, iter, mem, ops::{Index, IndexMut} + cmp::min, + collections::VecDeque, + iter, mem, + ops::{Index, IndexMut}, }; use arrow::array::{ArrayRef, BooleanArray}; @@ -210,6 +213,14 @@ impl Blocks { } } + pub fn iter_mut(&mut self) -> Box + '_> { + match self { + Blocks::Single(None) => Box::new(iter::empty()), + Blocks::Single(Some(single)) => Box::new(iter::once(single)), + Blocks::Multiple(multiple) => Box::new(multiple.iter_mut()), + } + } + pub fn clear(&mut self) { *self = Self::new(); } @@ -236,7 +247,9 @@ impl Index for Blocks { single } Blocks::Multiple(multiple) => &multiple[index], - Blocks::Single(None) => unreachable!("can't use index to access empty blocks"), + Blocks::Single(None) => { + unreachable!("can't use index to access empty blocks") + } } } } @@ -249,7 +262,9 @@ impl IndexMut for Blocks { single } Blocks::Multiple(multiple) => &mut multiple[index], - Blocks::Single(None) => unreachable!("can't use index to access empty blocks"), + Blocks::Single(None) => { + unreachable!("can't use index to access empty blocks") + } } } } diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index c67028892ba6..100f8e7ccaf5 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -34,7 +34,9 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; -use datafusion_expr_common::groups_accumulator::{Blocks, EmitTo, GroupsAccumulator, VecBlocks}; +use datafusion_expr_common::groups_accumulator::{ + Blocks, EmitTo, GroupsAccumulator, VecBlocks, +}; use datafusion_expr_common::{ accumulator::Accumulator, groups_accumulator::GroupStatesMode, }; @@ -436,6 +438,10 @@ pub fn ensure_enough_room_for_values( match mode { // It flat mode, we just a single builder, and grow it constantly. GroupStatesMode::Flat => { + if values.num_blocks() == 0 { + values.push_block(Vec::new()); + } + values .current_mut() .unwrap() @@ -444,12 +450,14 @@ pub fn ensure_enough_room_for_values( // It blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = { + let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { let cur_blk_idx = values.num_blocks() - 1; let exist_slots = (values.num_blocks() - 1) * blk_size + values.current().unwrap().len(); (cur_blk_idx, exist_slots) + } else { + (0, 0) }; // No new groups, don't need to expand, just return. @@ -457,8 +465,8 @@ pub fn ensure_enough_room_for_values( return; } - let exist_blks = values.num_blocks(); // Ensure blks are enough. + let exist_blks = values.num_blocks(); let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; if new_blks > 0 { for _ in 0..new_blks { 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 42500cdab08e..387e65849af7 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,14 +19,12 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator -use std::collections::VecDeque; - use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::{ - BlockedGroupIndex, EmitTo, GroupStatesMode, + BlockedGroupIndex, Blocks, EmitTo, GroupStatesMode, }; /// 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. @@ -368,7 +366,7 @@ pub struct BlockedNullState { /// /// If `seen_values[i]` is false, have not seen any values that /// pass the filter yet for group `i` - seen_values_blocks: VecDeque, + seen_values_blocks: Blocks, mode: GroupStatesMode, } @@ -382,7 +380,7 @@ impl Default for BlockedNullState { impl BlockedNullState { pub fn new(mode: GroupStatesMode) -> Self { Self { - seen_values_blocks: VecDeque::new(), + seen_values_blocks: Blocks::new(), mode, } } @@ -428,7 +426,7 @@ impl BlockedNullState { for (&group_index, &new_value) in iter { match self.mode { GroupStatesMode::Flat => seen_values_blocks - .back_mut() + .current_mut() .unwrap() .set_bit(group_index, true), GroupStatesMode::Blocked(_) => { @@ -468,7 +466,7 @@ impl BlockedNullState { if is_valid { match self.mode { GroupStatesMode::Flat => seen_values_blocks - .back_mut() + .current_mut() .unwrap() .set_bit(group_index, true), GroupStatesMode::Blocked(_) => { @@ -499,7 +497,7 @@ impl BlockedNullState { if is_valid { match self.mode { GroupStatesMode::Flat => seen_values_blocks - .back_mut() + .current_mut() .unwrap() .set_bit(group_index, true), GroupStatesMode::Blocked(_) => { @@ -527,7 +525,7 @@ impl BlockedNullState { if let Some(true) = filter_value { match self.mode { GroupStatesMode::Flat => seen_values_blocks - .back_mut() + .current_mut() .unwrap() .set_bit(group_index, true), GroupStatesMode::Blocked(_) => { @@ -556,7 +554,7 @@ impl BlockedNullState { if let Some(new_value) = new_value { match self.mode { GroupStatesMode::Flat => seen_values_blocks - .back_mut() + .current_mut() .unwrap() .set_bit(group_index, true), GroupStatesMode::Blocked(_) => { @@ -576,18 +574,18 @@ impl BlockedNullState { /// Similar as [NullState::build] but support the blocked version accumulator pub fn build(&mut self, emit_to: EmitTo) -> NullBuffer { - if self.seen_values_blocks.is_empty() { + if self.seen_values_blocks.num_blocks() == 0 { return NullBuffer::new(BooleanBufferBuilder::new(0).finish()); } let nulls = match emit_to { EmitTo::All => match self.mode { GroupStatesMode::Flat => { - self.seen_values_blocks.back_mut().unwrap().finish() + self.seen_values_blocks.current_mut().unwrap().finish() } GroupStatesMode::Blocked(blk_size) => { - let total_num = (self.seen_values_blocks.len() - 1) * blk_size - + self.seen_values_blocks.back().unwrap().len(); + let total_num = (self.seen_values_blocks.num_blocks() - 1) * blk_size + + self.seen_values_blocks.current().unwrap().len(); let mut total_buffer = BooleanBufferBuilder::new(total_num); for blk in self.seen_values_blocks.iter_mut() { @@ -603,7 +601,7 @@ impl BlockedNullState { EmitTo::First(n) => { assert!(matches!(self.mode, GroupStatesMode::Flat)); - let blk = self.seen_values_blocks.back_mut().unwrap(); + let blk = self.seen_values_blocks.current_mut().unwrap(); // split off the first N values in seen_values // // TODO make this more efficient rather than two @@ -617,7 +615,7 @@ impl BlockedNullState { first_n_null } EmitTo::CurrentBlock(_) => { - let mut cur_blk = self.seen_values_blocks.pop_front().unwrap(); + let mut cur_blk = self.seen_values_blocks.pop_first_block().unwrap(); cur_blk.finish() } }; @@ -735,7 +733,7 @@ fn initialize_builder( /// Similar as the [initialize_builder] but supported the blocked version accumulator fn ensure_enough_room_for_nulls( - builder_blocks: &mut VecDeque, + builder_blocks: &mut Blocks, mode: GroupStatesMode, total_num_groups: usize, default_value: bool, @@ -747,11 +745,11 @@ fn ensure_enough_room_for_nulls( match mode { // It flat mode, we just a single builder, and grow it constantly. GroupStatesMode::Flat => { - if builder_blocks.is_empty() { - builder_blocks.push_back(BooleanBufferBuilder::new(0)); + if builder_blocks.num_blocks() == 0 { + builder_blocks.push_block(BooleanBufferBuilder::new(0)); } - let builder = builder_blocks.back_mut().unwrap(); + let builder = builder_blocks.current_mut().unwrap(); if builder.len() < total_num_groups { let new_groups = total_num_groups - builder.len(); builder.append_n(new_groups, default_value); @@ -760,37 +758,45 @@ fn ensure_enough_room_for_nulls( // It blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if !builder_blocks.is_empty() { - let cur_blk_idx = builder_blocks.len() - 1; - let exist_slots = (builder_blocks.len() - 1) * blk_size - + builder_blocks.back().unwrap().len(); + let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { + let cur_blk_idx = builder_blocks.num_blocks() - 1; + let exist_slots = (builder_blocks.num_blocks() - 1) * blk_size + + builder_blocks.current().unwrap().len(); (cur_blk_idx, exist_slots) } else { (0, 0) }; - let exist_blks = builder_blocks.len(); - // Ensure blks are enough. + // No new groups, don't need to expand, just return. + if exist_slots >= total_num_groups { + return; + } + + // Ensure blks are enough + let exist_blks = builder_blocks.num_blocks(); let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - builder_blocks.reserve(new_blks); - for _ in 0..new_blks { - builder_blocks.push_back(BooleanBufferBuilder::new(blk_size)); + if new_blks > 0 { + for _ in 0..new_blks { + builder_blocks.push_block(BooleanBufferBuilder::new(blk_size)); + } } // Ensure slots are enough. let mut new_slots = total_num_groups - exist_slots; + // Expand current blk. let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); if cur_blk_rest_slots >= new_slots { builder_blocks[cur_blk_idx].append_n(new_slots, default_value); return; - } else { - builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; } + // Expand current blk to full, and expand next blks + builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + // Expand blks let expand_blks = new_slots / blk_size; for _ in 0..expand_blks { @@ -800,10 +806,12 @@ fn ensure_enough_room_for_nulls( // Expand the last blk. let last_expand_slots = new_slots % blk_size; - builder_blocks - .back_mut() - .unwrap() - .append_n(last_expand_slots, default_value); + if last_expand_slots > 0 { + builder_blocks + .current_mut() + .unwrap() + .append_n(last_expand_slots, default_value); + } } } } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index afc836f56448..c32dfbcdbbcd 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,7 +29,9 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks}; +use datafusion_expr::groups_accumulator::{ + BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks, +}; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 50e855025481..6e195249e77f 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,7 +16,9 @@ // under the License. use ahash::RandomState; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks}; +use datafusion_expr::groups_accumulator::{ + BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks, +}; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; use std::collections::HashSet; From 6de7fc72f4b2cb95cfcf6620d888e89fd60dc1b6 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 20:37:24 +0800 Subject: [PATCH 021/107] use `Blocks` in `GroupValuesRows`. --- .../expr-common/src/groups_accumulator.rs | 2 +- .../src/aggregates/group_values/row.rs | 35 +++++++++---------- 2 files changed, 18 insertions(+), 19 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 5adc02b8af13..9db7e9134fa1 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -269,7 +269,7 @@ impl IndexMut for Blocks { } } -impl Default for VecBlocks { +impl Default for Blocks { fn default() -> Self { Self::new() } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index c244ad414ad6..a7cf9904731e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. -use std::collections::VecDeque; use std::mem; use crate::aggregates::group_values::GroupValues; @@ -28,7 +27,7 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; -use datafusion_expr::groups_accumulator::GroupStatesMode; +use datafusion_expr::groups_accumulator::{Blocks, GroupStatesMode}; use datafusion_expr::EmitTo; use hashbrown::raw::RawTable; @@ -61,7 +60,7 @@ pub struct GroupValuesRows { /// important for multi-column group keys. /// /// [`Row`]: arrow::row::Row - group_values: VecDeque, + group_values: Blocks, /// reused buffer to store hashes hashes_buffer: Vec, @@ -97,7 +96,7 @@ impl GroupValuesRows { row_converter, map, map_size: 0, - group_values: VecDeque::new(), + group_values: Blocks::new(), hashes_buffer: Default::default(), rows_buffer, random_state: Default::default(), @@ -115,7 +114,7 @@ impl GroupValues for GroupValuesRows { let n_rows = group_rows.num_rows(); let mut group_values = mem::take(&mut self.group_values); - if group_values.is_empty() { + if group_values.num_blocks() == 0 { let block = match self.mode { GroupStatesMode::Flat => self.row_converter.empty_rows(0, 0), GroupStatesMode::Blocked(blk_size) => { @@ -123,7 +122,7 @@ impl GroupValues for GroupValuesRows { } }; - group_values.push_back(block); + group_values.push_block(block); } // tracks to which group each of the input rows belongs @@ -152,7 +151,7 @@ impl GroupValues for GroupValuesRows { match mode { GroupStatesMode::Flat => { group_rows.row(row) - == group_values.back().unwrap().row(*group_idx) + == group_values.current().unwrap().row(*group_idx) } GroupStatesMode::Blocked(_) => { let block_id = @@ -172,24 +171,24 @@ impl GroupValues for GroupValuesRows { // Add new entry to aggr_state and save newly created index let group_idx = match mode { GroupStatesMode::Flat => { - let blk = group_values.back_mut().unwrap(); + let blk = group_values.current_mut().unwrap(); let group_idx = blk.num_rows(); blk.push(group_rows.row(row)); group_idx } GroupStatesMode::Blocked(blk_size) => { - if group_values.back().unwrap().num_rows() == blk_size { + if group_values.current().unwrap().num_rows() == blk_size { // Use blk_size as offset cap, // and old block's buffer size as buffer cap let new_buf_cap = - rows_buffer_size(group_values.back().unwrap()); + rows_buffer_size(group_values.current().unwrap()); let new_blk = self.row_converter.empty_rows(blk_size, new_buf_cap); - group_values.push_back(new_blk); + group_values.push_block(new_blk); } - let blk_id = (group_values.len() - 1) as u64; - let cur_blk = group_values.back_mut().unwrap(); + let blk_id = (group_values.num_blocks() - 1) as u64; + let cur_blk = group_values.current_mut().unwrap(); let blk_offset = cur_blk.num_rows() as u64; cur_blk.push(group_rows.row(row)); @@ -246,7 +245,7 @@ impl GroupValues for GroupValuesRows { let mut output = match emit_to { EmitTo::All => match self.mode { GroupStatesMode::Flat => { - let blk = group_values.pop_back().unwrap(); + let blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(blk.into_iter())?; output } @@ -263,7 +262,7 @@ impl GroupValues for GroupValuesRows { let mut total_rows = self .row_converter .empty_rows(total_rows_num, total_buffer_size); - for rows in &group_values { + for rows in group_values.iter() { for row in rows.into_iter() { total_rows.push(row); } @@ -277,7 +276,7 @@ impl GroupValues for GroupValuesRows { } }, EmitTo::First(n) => { - let blk = group_values.back_mut().unwrap(); + let blk = group_values.current_mut().unwrap(); let groups_rows = blk.iter().take(n); let output = self.row_converter.convert_rows(groups_rows)?; // Clear out first n group keys by copying them to a new Rows. @@ -303,7 +302,7 @@ impl GroupValues for GroupValuesRows { output } EmitTo::CurrentBlock(true) => { - let cur_blk = group_values.pop_front().unwrap(); + let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; unsafe { for bucket in self.map.iter() { @@ -327,7 +326,7 @@ impl GroupValues for GroupValuesRows { output } EmitTo::CurrentBlock(false) => { - let cur_blk = group_values.pop_front().unwrap(); + let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; output } From 1567057d3cc8e411d43c0ae7f939f98b0464465e Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 21:18:56 +0800 Subject: [PATCH 022/107] use debug_assert instead of assert. --- datafusion/expr-common/src/groups_accumulator.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 9db7e9134fa1..c2ad105f2eaf 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -243,7 +243,7 @@ impl Index for Blocks { fn index(&self, index: usize) -> &T { match self { Blocks::Single(Some(single)) => { - assert!(index == 0); + debug_assert!(index == 0); single } Blocks::Multiple(multiple) => &multiple[index], @@ -258,7 +258,7 @@ impl IndexMut for Blocks { fn index_mut(&mut self, index: usize) -> &mut T { match self { Blocks::Single(Some(single)) => { - assert!(index == 0); + debug_assert!(index == 0); single } Blocks::Multiple(multiple) => &mut multiple[index], From a780401994d546d88c7dddb67c05df5335342453 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 23:33:54 +0800 Subject: [PATCH 023/107] refactor `Blocks`. --- .../expr-common/src/groups_accumulator.rs | 171 +++++++++--------- 1 file changed, 85 insertions(+), 86 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index c2ad105f2eaf..084e1d02c6f0 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -141,83 +141,93 @@ impl BlockedGroupIndex { } } -pub enum Blocks { - Single(Option), - Multiple(VecDeque), +pub struct Blocks { + /// The current block, it should be pushed into `previous` + /// when next block is pushed + current: Option, + + /// + previous: VecDeque, } impl Blocks { pub fn new() -> Self { - Self::Single(None) + Self { + current: None, + previous: VecDeque::new(), + } } pub fn current(&self) -> Option<&T> { - match self { - Blocks::Single(blk) => blk.as_ref(), - Blocks::Multiple(blks) => blks.back(), - } + self.current.as_ref() } pub fn current_mut(&mut self) -> Option<&mut T> { - match self { - Blocks::Single(blk) => blk.as_mut(), - Blocks::Multiple(blks) => blks.back_mut(), - } + self.current.as_mut() } pub fn push_block(&mut self, block: T) { - loop { - match self { - // If found it is Single, convert to Multiple first - Blocks::Single(single_opt) => { - let single_opt = mem::take(single_opt); - if single_opt.is_none() { - *self = Self::Single(Some(block)); - break; - } + // If empty, use the block as initialized current + if self.current.is_none() { + self.current = Some(block); + return; + } - let mut new_multiple = VecDeque::with_capacity(2); - new_multiple.push_back(single_opt.unwrap()); + // Take and push the old current to `previous` + let old_cur = std::mem::take(&mut self.current).unwrap(); + self.previous.push_back(old_cur); - *self = Self::Multiple(new_multiple); - } - // If found it Multiple, just push the block into it - Blocks::Multiple(multiple) => { - multiple.push_back(block); - break; - } - } - } + // Set the new current + self.current = Some(block); } pub fn pop_first_block(&mut self) -> Option { - match self { - Blocks::Single(single) => mem::take(single), - Blocks::Multiple(multiple) => multiple.pop_front(), + // If `previous` not empty, pop the first of them + if !self.previous.is_empty() { + return self.previous.pop_front(); } + + // Otherwise, we pop the current + std::mem::take(&mut self.current) } pub fn num_blocks(&self) -> usize { - match self { - Blocks::Single(None) => 0, - Blocks::Single(Some(_)) => 1, - Blocks::Multiple(multiple) => multiple.len(), + if self.current.is_none() { + return 0; } + + self.previous.len() + 1 } pub fn iter(&self) -> Box + '_> { - match self { - Blocks::Single(None) => Box::new(iter::empty()), - Blocks::Single(Some(single)) => Box::new(iter::once(single)), - Blocks::Multiple(multiple) => Box::new(multiple.iter()), + // If current is None, it means no data, return empty iter + if self.current.is_none() { + return Box::new(iter::empty()); + } + + let cur_iter = iter::once(self.current.as_ref().unwrap()); + + if !self.previous.is_empty() { + let previous_iter = self.previous.iter(); + Box::new(previous_iter.chain(cur_iter)) + } else { + Box::new(cur_iter) } } pub fn iter_mut(&mut self) -> Box + '_> { - match self { - Blocks::Single(None) => Box::new(iter::empty()), - Blocks::Single(Some(single)) => Box::new(iter::once(single)), - Blocks::Multiple(multiple) => Box::new(multiple.iter_mut()), + // If current is None, it means no data, return empty iter + if self.current.is_none() { + return Box::new(iter::empty()); + } + + let cur_iter = iter::once(self.current.as_mut().unwrap()); + + if !self.previous.is_empty() { + let previous_iter = self.previous.iter_mut(); + Box::new(previous_iter.chain(cur_iter)) + } else { + Box::new(cur_iter) } } @@ -227,13 +237,11 @@ impl Blocks { } impl fmt::Debug for Blocks { - fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - match self { - Self::Single(single) => f.debug_tuple("Single").field(single).finish(), - Self::Multiple(multiple) => { - f.debug_tuple("Multiple").field(multiple).finish() - } - } + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Blocks") + .field("current", &self.current) + .field("previous", &self.previous) + .finish() } } @@ -241,30 +249,20 @@ impl Index for Blocks { type Output = T; fn index(&self, index: usize) -> &T { - match self { - Blocks::Single(Some(single)) => { - debug_assert!(index == 0); - single - } - Blocks::Multiple(multiple) => &multiple[index], - Blocks::Single(None) => { - unreachable!("can't use index to access empty blocks") - } + if index < self.previous.len() { + &self.previous[index] + } else { + self.current.as_ref().unwrap() } } } impl IndexMut for Blocks { fn index_mut(&mut self, index: usize) -> &mut T { - match self { - Blocks::Single(Some(single)) => { - debug_assert!(index == 0); - single - } - Blocks::Multiple(multiple) => &mut multiple[index], - Blocks::Single(None) => { - unreachable!("can't use index to access empty blocks") - } + if index < self.previous.len() { + &mut self.previous[index] + } else { + self.current.as_mut().unwrap() } } } @@ -278,24 +276,25 @@ impl Default for Blocks { pub type VecBlocks = Blocks>; impl VecBlocks { - pub fn into_to_vec(self) -> Vec { - match self { - Blocks::Single(None) => Vec::new(), - Blocks::Single(Some(single)) => single, - Blocks::Multiple(multiple) => { - multiple.into_iter().flat_map(|v| v.into_iter()).collect() - } + pub fn into_to_vec(mut self) -> Vec { + if self.current.is_none() { + return Vec::new(); + } + + if self.previous.is_empty() { + self.current.unwrap() + } else { + self.iter_mut() + .flat_map(|blk| std::mem::take(blk).into_iter()) + .collect::>() } } pub fn capacity(&self) -> usize { - match self { - Blocks::Single(None) => 0, - Blocks::Single(Some(single)) => single.capacity(), - Blocks::Multiple(multiple) => { - multiple.iter().map(|blk| blk.capacity()).sum::() - } - } + let cur_cap = self.current.as_ref().map(|blk| blk.capacity()).unwrap_or(0); + let prev_cap = self.previous.iter().map(|p| p.capacity()).sum::(); + + cur_cap + prev_cap } } From 5fc7bf544cdcc45f8d9dde46e376774d3979daa1 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 18 Aug 2024 23:56:44 +0800 Subject: [PATCH 024/107] rename `CurrentBlock` to `NextBlock`, and add more comments. --- .../expr-common/src/groups_accumulator.rs | 17 ++++++++++++----- .../aggregate/groups_accumulator/accumulate.rs | 4 ++-- .../src/aggregate/groups_accumulator/bool_op.rs | 2 +- .../src/aggregates/group_values/bytes.rs | 2 +- .../src/aggregates/group_values/bytes_view.rs | 2 +- .../src/aggregates/group_values/primitive.rs | 2 +- .../src/aggregates/group_values/row.rs | 4 ++-- .../physical-plan/src/aggregates/row_hash.rs | 4 ++-- 8 files changed, 22 insertions(+), 15 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 084e1d02c6f0..b3e4e6cd8400 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -39,8 +39,14 @@ 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 all groups managed by blocks - CurrentBlock(bool), + /// Emit next block in the blocked managed groups + /// + /// The flag's meaning: + /// - `true` represents it will be added new groups again, + /// we don't need to shift the values down. + /// - `false` represents new groups still be possible to be + /// added, and we need to shift the values down. + NextBlock(bool), } impl EmitTo { @@ -62,7 +68,7 @@ impl EmitTo { std::mem::swap(v, &mut t); t } - Self::CurrentBlock(_) => unreachable!( + Self::NextBlock(_) => unreachable!( "can not support blocked emission in take_needed, you should use take_needed_from_blocks" ), } @@ -112,7 +118,7 @@ impl EmitTo { } } } - Self::CurrentBlock(_) => blocks.pop_first_block().unwrap(), + Self::NextBlock(_) => blocks.pop_first_block().unwrap(), } } } @@ -141,12 +147,13 @@ impl BlockedGroupIndex { } } +/// The basic data structure for blocked aggregation intermediate results pub struct Blocks { /// The current block, it should be pushed into `previous` /// when next block is pushed current: Option, - /// + /// The previous: VecDeque, } 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 387e65849af7..fe374ba2805b 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -348,7 +348,7 @@ impl NullState { } first_n_null } - EmitTo::CurrentBlock(_) => { + EmitTo::NextBlock(_) => { unreachable!("can't support blocked emission in flat NullState") } }; @@ -614,7 +614,7 @@ impl BlockedNullState { } first_n_null } - EmitTo::CurrentBlock(_) => { + EmitTo::NextBlock(_) => { let mut cur_blk = self.seen_values_blocks.pop_first_block().unwrap(); cur_blk.finish() } 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 d205fc2dc901..24e9249e8687 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,7 +117,7 @@ where } first_n } - EmitTo::CurrentBlock(_) => { + EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( "blocked group values management is not supported".to_string(), )) diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs index e2fb47722cb8..4e47d269cf50 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs @@ -116,7 +116,7 @@ impl GroupValues for GroupValuesByes { emit_group_values } - EmitTo::CurrentBlock(_) => { + EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( "blocked group values is not supported yet".to_string(), )) diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs index 82bdd76e43d0..f2580443880c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs @@ -117,7 +117,7 @@ impl GroupValues for GroupValuesBytesView { emit_group_values } - EmitTo::CurrentBlock(_) => { + EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( "blocked group values is not supported yet".to_string(), )) diff --git a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs index ba63c8eb5733..18646f23fbc4 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs @@ -206,7 +206,7 @@ where std::mem::swap(&mut self.values, &mut split); build_primitive(split, null_group) } - EmitTo::CurrentBlock(_) => { + EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( "blocked group values is not supported yet".to_string(), )) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index a7cf9904731e..34d1543949a0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -301,7 +301,7 @@ impl GroupValues for GroupValuesRows { } output } - EmitTo::CurrentBlock(true) => { + EmitTo::NextBlock(true) => { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; unsafe { @@ -325,7 +325,7 @@ impl GroupValues for GroupValuesRows { } output } - EmitTo::CurrentBlock(false) => { + EmitTo::NextBlock(false) => { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; output diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index eb6dc266fc5e..52f5f51d5c7c 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -772,7 +772,7 @@ impl Stream for GroupedHashAggregateStream { self.exec_state = ExecutionState::ProducingBlocks(Some(*blk - 1)); - EmitTo::CurrentBlock(true) + EmitTo::NextBlock(true) } else { self.exec_state = if self.input_done { ExecutionState::Done @@ -784,7 +784,7 @@ impl Stream for GroupedHashAggregateStream { continue; } } else { - EmitTo::CurrentBlock(false) + EmitTo::NextBlock(false) }; let emit_result = self.emit(emit_to, false); From e143c5910fde79bd78ffa804de3755630f038baf Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 00:43:11 +0800 Subject: [PATCH 025/107] minor optimization. --- .../expr-common/src/groups_accumulator.rs | 7 ++----- .../src/aggregates/group_values/row.rs | 20 +++++++++++++++---- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index b3e4e6cd8400..2a81737f63d7 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -180,12 +180,9 @@ impl Blocks { return; } - // Take and push the old current to `previous` - let old_cur = std::mem::take(&mut self.current).unwrap(); + // Take and push the old current to `previous`, use the b + let old_cur = std::mem::replace(&mut self.current, Some(block)).unwrap(); self.previous.push_back(old_cur); - - // Set the new current - self.current = Some(block); } pub fn pop_first_block(&mut self) -> Option { diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 34d1543949a0..49c7caa26ee8 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -233,10 +233,22 @@ impl GroupValues for GroupValuesRows { } fn len(&self) -> usize { - self.group_values - .iter() - .map(|rows| rows.num_rows()) - .sum::() + match self.mode { + GroupStatesMode::Flat => self + .group_values + .current() + .map(|g| g.num_rows()) + .unwrap_or(0), + GroupStatesMode::Blocked(blk_size) => { + let num_blocks = self.group_values.num_blocks(); + if num_blocks == 0 { + return 0; + } + + (num_blocks - 1) * blk_size + + self.group_values.current().unwrap().num_rows() + } + } } fn emit(&mut self, emit_to: EmitTo) -> Result> { From 5fb1748700408b5140269789a12c167ceecafb34 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 00:47:27 +0800 Subject: [PATCH 026/107] fix comments. --- datafusion/expr-common/src/groups_accumulator.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 2a81737f63d7..61c55b30f030 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -180,7 +180,8 @@ impl Blocks { return; } - // Take and push the old current to `previous`, use the b + // Take and push the old current to `previous`, + // use input `block` as the new `current` let old_cur = std::mem::replace(&mut self.current, Some(block)).unwrap(); self.previous.push_back(old_cur); } From 564e6d33b6cc32ba6e2b06218c1b98df5e70813f Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 01:00:13 +0800 Subject: [PATCH 027/107] add todos. --- datafusion/expr-common/src/groups_accumulator.rs | 4 ++++ datafusion/physical-plan/src/aggregates/group_values/row.rs | 2 ++ 2 files changed, 6 insertions(+) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 61c55b30f030..26d0b215abcd 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -204,6 +204,8 @@ impl Blocks { self.previous.len() + 1 } + // TODO: maybe impl a specific Iterator rather than use the trait object, + // it can slightly improve performance by eliminating the dynamic dispatch. pub fn iter(&self) -> Box + '_> { // If current is None, it means no data, return empty iter if self.current.is_none() { @@ -220,6 +222,8 @@ impl Blocks { } } + // TODO: maybe impl a specific Iterator rather than use the trait object, + // it can slightly improve performance by eliminating the dynamic dispatch. pub fn iter_mut(&mut self) -> Box + '_> { // If current is None, it means no data, return empty iter if self.current.is_none() { diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 49c7caa26ee8..bb2050bbb4eb 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -216,6 +216,8 @@ impl GroupValues for GroupValuesRows { } fn size(&self) -> usize { + // TODO: support size stats in `Blocks`, + // it is too expansive to calculate it again and again. let group_values_size = self .group_values .iter() From 7e607ebb87ef41fc570845835ed7412f72b382df Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 01:13:53 +0800 Subject: [PATCH 028/107] reduce repeated codes. --- .../expr-common/src/groups_accumulator.rs | 14 ++++++- .../src/aggregates/group_values/row.rs | 39 +++++++++---------- 2 files changed, 32 insertions(+), 21 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 26d0b215abcd..5d0cc164a60e 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -145,6 +145,18 @@ impl BlockedGroupIndex { block_offset, } } + + pub fn new_from_parts(block_id: usize, block_offset: usize) -> Self { + Self { + block_id, + block_offset, + } + } + + pub fn as_packed_index(&self) -> usize { + ((((self.block_id as u64) << 32) & 0xffffffff00000000) + | (self.block_offset as u64 & 0x00000000ffffffff)) as usize + } } /// The basic data structure for blocked aggregation intermediate results @@ -180,7 +192,7 @@ impl Blocks { return; } - // Take and push the old current to `previous`, + // Take and push the old current to `previous`, // use input `block` as the new `current` let old_cur = std::mem::replace(&mut self.current, Some(block)).unwrap(); self.previous.push_back(old_cur); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index bb2050bbb4eb..a41ef2e3ac0e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -27,7 +27,7 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; -use datafusion_expr::groups_accumulator::{Blocks, GroupStatesMode}; +use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode}; use datafusion_expr::EmitTo; use hashbrown::raw::RawTable; @@ -154,11 +154,10 @@ impl GroupValues for GroupValuesRows { == group_values.current().unwrap().row(*group_idx) } GroupStatesMode::Blocked(_) => { - let block_id = - ((*group_idx as u64 >> 32) & 0x00000000ffffffff) as usize; - let block_offset = - ((*group_idx as u64) & 0x00000000ffffffff) as usize; - group_rows.row(row) == group_values[block_id].row(block_offset) + let blocked_index = BlockedGroupIndex::new(*group_idx); + group_rows.row(row) + == group_values[blocked_index.block_id] + .row(blocked_index.block_offset) } } }); @@ -187,14 +186,14 @@ impl GroupValues for GroupValuesRows { group_values.push_block(new_blk); } - let blk_id = (group_values.num_blocks() - 1) as u64; + let blk_id = group_values.num_blocks() - 1; let cur_blk = group_values.current_mut().unwrap(); - let blk_offset = cur_blk.num_rows() as u64; + let blk_offset = cur_blk.num_rows(); cur_blk.push(group_rows.row(row)); - (((blk_id << 32) & 0xffffffff00000000) - | (blk_offset & 0x00000000ffffffff)) - as usize + let blocked_index = + BlockedGroupIndex::new_from_parts(blk_id, blk_offset); + blocked_index.as_packed_index() } }; @@ -321,16 +320,16 @@ impl GroupValues for GroupValuesRows { unsafe { for bucket in self.map.iter() { // Decrement group index by n - let group_idx = bucket.as_ref().1 as u64; - let blk_id = (group_idx >> 32) & 0x00000000ffffffff; - let blk_offset = group_idx & 0x00000000ffffffff; - match blk_id.checked_sub(1) { + let group_idx = bucket.as_ref().1; + let old_blk_idx = BlockedGroupIndex::new(group_idx); + match old_blk_idx.block_id.checked_sub(1) { // Group index was >= n, shift value down - Some(bid) => { - let new_group_idx = (((bid << 32) & 0xffffffff00000000) - | (blk_offset & 0x00000000ffffffff)) - as usize; - bucket.as_mut().1 = new_group_idx; + Some(new_blk_id) => { + let new_group_idx = BlockedGroupIndex::new_from_parts( + new_blk_id, + old_blk_idx.block_offset, + ); + bucket.as_mut().1 = new_group_idx.as_packed_index(); } // Group index was < n, so remove from table None => self.map.erase(bucket), From ec9bf215109d0c8ae82fcdb2581a94aa66ca0502 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 22:19:12 +0800 Subject: [PATCH 029/107] disable blocked optimization in spilling case, and add comments. --- .../physical-plan/src/aggregates/row_hash.rs | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 52f5f51d5c7c..a42ae07ba93e 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -42,7 +42,7 @@ use arrow_schema::SortOptions; use datafusion_common::{internal_datafusion_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, MemoryPool, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; use datafusion_expr::groups_accumulator::GroupStatesMode; @@ -558,6 +558,7 @@ impl GroupedHashAggregateStream { // Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. let enable_blocked_group_states = maybe_enable_blocked_group_states( + &context, group_values.as_mut(), &mut accumulators, batch_size, @@ -593,15 +594,21 @@ impl GroupedHashAggregateStream { /// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. /// The blocked optimization will be enabled when: /// - It is not streaming aggregation(because blocked mode can't support Emit::first(exact n)) -/// - The accumulator is not empty +/// - 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` and all `GroupsAccumulator`s support blocked mode +// TODO: support blocked optimization in streaming, spilling, and maybe empty accumulators case? fn maybe_enable_blocked_group_states( + context: &TaskContext, group_values: &mut dyn GroupValues, accumulators: &mut [Box], block_size: usize, group_ordering: &GroupOrdering, ) -> Result { - if !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() { + if !matches!(group_ordering, GroupOrdering::None) + || accumulators.is_empty() + || enable_spilling(context.memory_pool().as_ref()) + { return Ok(false); } @@ -621,6 +628,11 @@ fn maybe_enable_blocked_group_states( } } +// TODO: we should add a function(like `name`) to distinguish different memory pools. +fn enable_spilling(memory_pool: &dyn MemoryPool) -> bool { + !format!("{memory_pool:?}").contains("UnboundedMemoryPool") +} + /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if /// that is supported by the aggregate, or a /// [`GroupsAccumulatorAdapter`] if not. From cb8da87a77f256d7b0f6107a9af88768635b82e1 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 22:37:26 +0800 Subject: [PATCH 030/107] add more comments and remove stale codes. --- .../expr-common/src/groups_accumulator.rs | 51 +++++++++---------- .../src/aggregates/group_values/mod.rs | 4 ++ 2 files changed, 27 insertions(+), 28 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 5d0cc164a60e..2c76a5dbb9fe 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -28,6 +28,9 @@ use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; use std::fmt; +const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; +const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; + /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy)] pub enum EmitTo { @@ -78,14 +81,9 @@ impl EmitTo { /// returning a `Vec` with elements taken. /// /// The detailed behavior in different emissions: - /// - For Emit::All, the groups in blocks will be merged into - /// single block and returned. - /// - /// - For Emit::First, it will be only supported in the flat, - /// similar as `take_needed`. - /// /// - For Emit::CurrentBlock, the first block will be taken and return. - /// + /// - For Emit::All and Emit::First, it will be only supported in `GroupStatesMode::Flat`, + /// similar as `take_needed`. pub fn take_needed_from_blocks( &self, blocks: &mut VecBlocks, @@ -95,8 +93,7 @@ impl EmitTo { Self::All => match mode { GroupStatesMode::Flat => blocks.pop_first_block().unwrap(), GroupStatesMode::Blocked(_) => { - let blocks = mem::take(blocks); - blocks.into_to_vec() + unreachable!("can't support Emit::All in blocked mode accumulator"); } }, Self::First(n) => { @@ -129,6 +126,11 @@ pub enum GroupStatesMode { Blocked(usize), } +/// Blocked style group index used in blocked mode group values and accumulators +/// +/// Parts in index: +/// - High 32 bits represent `block_id` +/// - Low 32 bits represent `block_offset` #[derive(Debug, Clone, Copy)] pub struct BlockedGroupIndex { pub block_id: usize, @@ -137,8 +139,10 @@ pub struct BlockedGroupIndex { impl BlockedGroupIndex { pub fn new(group_index: usize) -> Self { - let block_id = ((group_index as u64 >> 32) & 0x00000000ffffffff) as usize; - let block_offset = ((group_index as u64) & 0x00000000ffffffff) as usize; + let block_id = + ((group_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + let block_offset = + ((group_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; Self { block_id, @@ -154,8 +158,9 @@ impl BlockedGroupIndex { } pub fn as_packed_index(&self) -> usize { - ((((self.block_id as u64) << 32) & 0xffffffff00000000) - | (self.block_offset as u64 & 0x00000000ffffffff)) as usize + ((((self.block_id as u64) << 32) & BLOCKED_INDEX_HIGH_32_BITS_MASK) + | (self.block_offset as u64 & BLOCKED_INDEX_LOW_32_BITS_MASK)) + as usize } } @@ -165,7 +170,7 @@ pub struct Blocks { /// when next block is pushed current: Option, - /// The + /// Previous blocks pushed before `current` previous: VecDeque, } @@ -297,20 +302,6 @@ impl Default for Blocks { pub type VecBlocks = Blocks>; impl VecBlocks { - pub fn into_to_vec(mut self) -> Vec { - if self.current.is_none() { - return Vec::new(); - } - - if self.previous.is_empty() { - self.current.unwrap() - } else { - self.iter_mut() - .flat_map(|blk| std::mem::take(blk).into_iter()) - .collect::>() - } - } - pub fn capacity(&self) -> usize { let cur_cap = self.current.as_ref().map(|blk| blk.capacity()).unwrap_or(0); let prev_cap = self.previous.iter().map(|p| p.capacity()).sum::(); @@ -406,10 +397,14 @@ pub trait GroupsAccumulator: Send { /// [`Accumulator::state`]: crate::accumulator::Accumulator::state fn state(&mut self, emit_to: EmitTo) -> Result>; + /// Returns `true` if this accumulator supports blocked mode. fn supports_blocked_mode(&self) -> bool { false } + /// Switch the accumulator to flat or blocked mode. + /// + /// After switching mode, all data in previous mode will be cleared. fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { return Err(DataFusionError::NotImplemented( diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index f8dc7c01ba12..9a5d10bd6256 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -53,10 +53,14 @@ pub trait GroupValues: Send { /// Clear the contents and shrink the capacity to the size of the batch (free up memory usage) fn clear_shrink(&mut self, batch: &RecordBatch); + /// Returns `true` if this group values supports blocked mode. fn supports_blocked_mode(&self) -> bool { false } + /// Switch the group values to flat or blocked mode. + /// + /// After switching mode, all data in previous mode will be cleared. fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { return Err(DataFusionError::NotImplemented( From e054c8b3f41418d446c2017a3770798403987189 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 22:42:33 +0800 Subject: [PATCH 031/107] not try to support spilling in blocked mode in currently. --- .../expr-common/src/groups_accumulator.rs | 42 ++++++++---------- .../groups_accumulator/accumulate.rs | 28 +++--------- .../src/aggregates/group_values/row.rs | 43 ++++++------------- .../physical-plan/src/aggregates/row_hash.rs | 12 ++---- 4 files changed, 41 insertions(+), 84 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 2c76a5dbb9fe..2d72facfa144 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -90,32 +90,26 @@ impl EmitTo { mode: GroupStatesMode, ) -> Vec { match self { - Self::All => match mode { - GroupStatesMode::Flat => blocks.pop_first_block().unwrap(), - GroupStatesMode::Blocked(_) => { - unreachable!("can't support Emit::All in blocked mode accumulator"); - } - }, + Self::All => { + debug_assert!(matches!(mode, GroupStatesMode::Flat)); + blocks.pop_first_block().unwrap() + } Self::First(n) => { - match mode { - GroupStatesMode::Flat => { - let block = blocks.current_mut().unwrap(); - let split_at = min(block.len(), *n); - - // get end n+1,.. values into t - let mut t = block.split_off(split_at); - // leave n+1,.. in v - std::mem::swap(block, &mut t); - t - } - GroupStatesMode::Blocked(_) => { - unreachable!( - "can't support Emit::First in blocked mode accumulator" - ); - } - } + debug_assert!(matches!(mode, GroupStatesMode::Flat)); + + let block = blocks.current_mut().unwrap(); + let split_at = min(block.len(), *n); + + // get end n+1,.. values into t + let mut t = block.split_off(split_at); + // leave n+1,.. in v + std::mem::swap(block, &mut t); + t + } + Self::NextBlock(_) => { + debug_assert!(matches!(mode, GroupStatesMode::Blocked(_))); + blocks.pop_first_block().unwrap() } - Self::NextBlock(_) => blocks.pop_first_block().unwrap(), } } } 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 fe374ba2805b..4963cc8a9634 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -579,33 +579,18 @@ impl BlockedNullState { } let nulls = match emit_to { - EmitTo::All => match self.mode { - GroupStatesMode::Flat => { - self.seen_values_blocks.current_mut().unwrap().finish() - } - GroupStatesMode::Blocked(blk_size) => { - let total_num = (self.seen_values_blocks.num_blocks() - 1) * blk_size - + self.seen_values_blocks.current().unwrap().len(); - let mut total_buffer = BooleanBufferBuilder::new(total_num); - - for blk in self.seen_values_blocks.iter_mut() { - let nulls = blk.finish(); - for seen in nulls.iter() { - total_buffer.append(seen); - } - } - - total_buffer.finish() - } - }, + EmitTo::All => { + debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); + self.seen_values_blocks.current_mut().unwrap().finish() + } EmitTo::First(n) => { - assert!(matches!(self.mode, GroupStatesMode::Flat)); + debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); - let blk = self.seen_values_blocks.current_mut().unwrap(); // split off the first N values in seen_values // // TODO make this more efficient rather than two // copies and bitwise manipulation + let blk = self.seen_values_blocks.current_mut().unwrap(); let nulls = blk.finish(); let first_n_null: BooleanBuffer = nulls.iter().take(n).collect(); // reset the existing seen buffer @@ -615,6 +600,7 @@ impl BlockedNullState { first_n_null } EmitTo::NextBlock(_) => { + debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); let mut cur_blk = self.seen_values_blocks.pop_first_block().unwrap(); cur_blk.finish() } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index a41ef2e3ac0e..adfa4c11791c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -256,39 +256,16 @@ impl GroupValues for GroupValuesRows { let mut group_values = mem::take(&mut self.group_values); let mut output = match emit_to { - EmitTo::All => match self.mode { - GroupStatesMode::Flat => { - let blk = group_values.pop_first_block().unwrap(); - let output = self.row_converter.convert_rows(blk.into_iter())?; - output - } - GroupStatesMode::Blocked(_) => { - let mut total_rows_num = 0; - let mut total_buffer_size = 0; - group_values.iter().for_each(|rows| { - let rows_num = rows.num_rows(); - let rows_buffer_size = rows_buffer_size(rows); - total_rows_num += rows_num; - total_buffer_size += rows_buffer_size; - }); - - let mut total_rows = self - .row_converter - .empty_rows(total_rows_num, total_buffer_size); - for rows in group_values.iter() { - for row in rows.into_iter() { - total_rows.push(row); - } - } + EmitTo::All => { + debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); - group_values.clear(); - - let output = - self.row_converter.convert_rows(total_rows.into_iter())?; - output - } - }, + let blk = group_values.pop_first_block().unwrap(); + let output = self.row_converter.convert_rows(blk.into_iter())?; + output + } EmitTo::First(n) => { + debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); + let blk = group_values.current_mut().unwrap(); let groups_rows = blk.iter().take(n); let output = self.row_converter.convert_rows(groups_rows)?; @@ -315,6 +292,8 @@ impl GroupValues for GroupValuesRows { output } EmitTo::NextBlock(true) => { + debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); + let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; unsafe { @@ -339,6 +318,8 @@ impl GroupValues for GroupValuesRows { output } EmitTo::NextBlock(false) => { + debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); + let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; output diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index a42ae07ba93e..45f9c57354ad 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -1045,15 +1045,11 @@ impl GroupedHashAggregateStream { && matches!(self.mode, AggregateMode::Partial) && self.update_memory_reservation().is_err() { - if !self.enable_blocked_group_states { - let n = self.group_values.len() / self.batch_size * self.batch_size; - let batch = self.emit(EmitTo::First(n), false)?; - self.exec_state = ExecutionState::ProducingOutput(batch); - } else { - let blocks = self.group_values.len() / self.group_states_block_size; - self.exec_state = ExecutionState::ProducingBlocks(Some(blocks)); - } + let n = self.group_values.len() / self.batch_size * self.batch_size; + let batch = self.emit(EmitTo::First(n), false)?; + self.exec_state = ExecutionState::ProducingOutput(batch); } + Ok(()) } From 478627d6faece9729fe000628f5446acccc9f7c2 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 23:04:18 +0800 Subject: [PATCH 032/107] improve error messages. --- datafusion/expr-common/src/groups_accumulator.rs | 2 +- datafusion/physical-plan/src/aggregates/group_values/bytes.rs | 2 +- .../physical-plan/src/aggregates/group_values/bytes_view.rs | 2 +- datafusion/physical-plan/src/aggregates/group_values/mod.rs | 2 +- .../physical-plan/src/aggregates/group_values/primitive.rs | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 2d72facfa144..6491d335f8d8 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -402,7 +402,7 @@ pub trait GroupsAccumulator: Send { fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { return Err(DataFusionError::NotImplemented( - "only flat mode accumulator is supported yet".to_string(), + "this accumulator doesn't support blocked mode yet".to_string(), )); } diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs index 4e47d269cf50..2a5d4da8bbc5 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes.rs @@ -118,7 +118,7 @@ impl GroupValues for GroupValuesByes { } EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( - "blocked group values is not supported yet".to_string(), + "this group values doesn't support blocked mode yet".to_string(), )) } }; diff --git a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs index f2580443880c..78f558112f83 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/bytes_view.rs @@ -119,7 +119,7 @@ impl GroupValues for GroupValuesBytesView { } EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( - "blocked group values is not supported yet".to_string(), + "this group values doesn't support blocked mode yet".to_string(), )) } }; diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 9a5d10bd6256..6c727b327afd 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -64,7 +64,7 @@ pub trait GroupValues: Send { fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { if matches!(&mode, GroupStatesMode::Blocked(_)) { return Err(DataFusionError::NotImplemented( - "only flat mode group values is supported yet".to_string(), + "this group values doesn't support blocked mode yet".to_string(), )); } diff --git a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs index 18646f23fbc4..5f35faed3640 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/primitive.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/primitive.rs @@ -208,7 +208,7 @@ where } EmitTo::NextBlock(_) => { return Err(DataFusionError::NotImplemented( - "blocked group values is not supported yet".to_string(), + "this group values doesn't support blocked mode yet".to_string(), )) } }; From ead0076421c011fa45b6b3aa4ad819f3e66ee4c2 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 23:16:26 +0800 Subject: [PATCH 033/107] add comment for `ProducingBlocks`. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 45f9c57354ad..0c514ef8bfc4 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -63,6 +63,15 @@ 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. + /// + /// An optional `blocks num` exists in it: + /// - `Some(n)` represents `n` blocks should be produced. + /// - `None` represents we should produce blocks until exhausted. + /// ProducingBlocks(Option), /// Produce intermediate aggregate state for each input row without /// aggregation. From db1adbe66106cc96ec0d9643ee586e88a7a59e22 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 23:25:11 +0800 Subject: [PATCH 034/107] add comments for GroupStatesMode. --- datafusion/expr-common/src/groups_accumulator.rs | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 6491d335f8d8..440ea4389cec 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -114,6 +114,16 @@ impl EmitTo { } } +/// Mode for `accumulators` and `group values` +/// +/// Their meanings: +/// - Flat, the values in them will be managed with a single `Vec`. +/// It will grow constantly when more and more values are inserted, +/// that leads to a considerable amount of copying, and finally a bad performance. +/// +/// - Blocked, the values in them will be managed with multiple `Vec`s. +/// When the block is large enough, a new block will be allocated and used +/// for inserting. Obviously, this strategy can avoid copying and get a good performance. #[derive(Debug, Clone, Copy)] pub enum GroupStatesMode { Flat, From 03189a773c82f4b45d7d670531911850a0e14e77 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 23:29:49 +0800 Subject: [PATCH 035/107] remove unused import. --- datafusion/expr-common/src/groups_accumulator.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 440ea4389cec..24a6a06e578b 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -20,7 +20,7 @@ use std::{ cmp::min, collections::VecDeque, - iter, mem, + iter, ops::{Index, IndexMut}, }; From 49c5e5e36f92925f5e82a5811301dee445e3c7ab Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 19 Aug 2024 23:32:11 +0800 Subject: [PATCH 036/107] fix clippy. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 3 --- 1 file changed, 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 0c514ef8bfc4..92517ab7dd2d 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -443,8 +443,6 @@ pub(crate) struct GroupedHashAggregateStream { /// The [`RuntimeEnv`] associated with the [`TaskContext`] argument runtime: Arc, enable_blocked_group_states: bool, - - group_states_block_size: usize, } impl GroupedHashAggregateStream { @@ -595,7 +593,6 @@ impl GroupedHashAggregateStream { group_values_soft_limit: agg.limit, skip_aggregation_probe, enable_blocked_group_states, - group_states_block_size: batch_size, }) } } From 50e895897d449eccdac5b0e05a0e12e69115b46e Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 20 Aug 2024 01:13:49 +0800 Subject: [PATCH 037/107] fix clippy. --- datafusion/expr-common/src/groups_accumulator.rs | 14 +++++++------- .../src/aggregates/group_values/row.rs | 6 ++---- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 24a6a06e578b..78bcb78f2be0 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -46,9 +46,9 @@ pub enum EmitTo { /// /// The flag's meaning: /// - `true` represents it will be added new groups again, - /// we don't need to shift the values down. + /// we don't need to shift the values down. /// - `false` represents new groups still be possible to be - /// added, and we need to shift the values down. + /// added, and we need to shift the values down. NextBlock(bool), } @@ -83,7 +83,7 @@ impl EmitTo { /// The detailed behavior in different emissions: /// - For Emit::CurrentBlock, the first block will be taken and return. /// - For Emit::All and Emit::First, it will be only supported in `GroupStatesMode::Flat`, - /// similar as `take_needed`. + /// similar as `take_needed`. pub fn take_needed_from_blocks( &self, blocks: &mut VecBlocks, @@ -118,12 +118,12 @@ impl EmitTo { /// /// Their meanings: /// - Flat, the values in them will be managed with a single `Vec`. -/// It will grow constantly when more and more values are inserted, -/// that leads to a considerable amount of copying, and finally a bad performance. +/// It will grow constantly when more and more values are inserted, +/// that leads to a considerable amount of copying, and finally a bad performance. /// /// - Blocked, the values in them will be managed with multiple `Vec`s. -/// When the block is large enough, a new block will be allocated and used -/// for inserting. Obviously, this strategy can avoid copying and get a good performance. +/// When the block is large enough, a new block will be allocated and used +/// for inserting. Obviously, this strategy can avoid copying and get a good performance. #[derive(Debug, Clone, Copy)] pub enum GroupStatesMode { Flat, diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index adfa4c11791c..4fd3501edfef 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -260,8 +260,7 @@ impl GroupValues for GroupValuesRows { debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); let blk = group_values.pop_first_block().unwrap(); - let output = self.row_converter.convert_rows(blk.into_iter())?; - output + self.row_converter.convert_rows(blk.into_iter())? } EmitTo::First(n) => { debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); @@ -321,8 +320,7 @@ impl GroupValues for GroupValuesRows { debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); let cur_blk = group_values.pop_first_block().unwrap(); - let output = self.row_converter.convert_rows(cur_blk.iter())?; - output + self.row_converter.convert_rows(cur_blk.iter())? } }; From 13e74b00d3860341cb30db754cf42b40d9546fb3 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 20 Aug 2024 01:36:25 +0800 Subject: [PATCH 038/107] add comments. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 1 + 1 file changed, 1 insertion(+) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 92517ab7dd2d..1c63827f91a3 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -442,6 +442,7 @@ pub(crate) struct GroupedHashAggregateStream { /// The [`RuntimeEnv`] associated with the [`TaskContext`] argument runtime: Arc, + /// Have we enabled the blocked optimization for group values and accumulators. enable_blocked_group_states: bool, } From f5684a0a2faeebcfaf5eb34d967a6aacc17f657e Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 20 Aug 2024 11:43:32 +0800 Subject: [PATCH 039/107] improve comments. --- datafusion/expr-common/src/groups_accumulator.rs | 16 +++++++++------- .../src/aggregate/groups_accumulator.rs | 2 +- .../aggregate/groups_accumulator/accumulate.rs | 6 +++--- .../src/aggregates/group_values/mod.rs | 1 + 4 files changed, 14 insertions(+), 11 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 78bcb78f2be0..32f684580e70 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -45,10 +45,10 @@ pub enum EmitTo { /// Emit next block in the blocked managed groups /// /// The flag's meaning: - /// - `true` represents it will be added new groups again, - /// we don't need to shift the values down. - /// - `false` represents new groups still be possible to be - /// added, and we need to shift the values down. + /// - `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), } @@ -121,9 +121,10 @@ impl EmitTo { /// It will grow constantly when more and more values are inserted, /// that leads to a considerable amount of copying, and finally a bad performance. /// -/// - Blocked, the values in them will be managed with multiple `Vec`s. -/// When the block is large enough, a new block will be allocated and used -/// for inserting. Obviously, this strategy can avoid copying and get a good performance. +/// - Blocked(block_size), the values in them will be managed with multiple `Vec`s. +/// When the block is large enough(reach block_size), a new block will be allocated +/// and used for inserting. +/// Obviously, this strategy can avoid copying and get a good performance. #[derive(Debug, Clone, Copy)] pub enum GroupStatesMode { Flat, @@ -407,6 +408,7 @@ pub trait GroupsAccumulator: Send { } /// Switch the accumulator to flat or blocked mode. + /// You can see detail about the mode on [GroupStatesMode]. /// /// After switching mode, all data in previous mode will be cleared. fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 100f8e7ccaf5..507e40d12302 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -447,7 +447,7 @@ pub fn ensure_enough_room_for_values( .unwrap() .resize(total_num_groups, default_value); } - // It blocked mode, we ensure the blks are enough first, + // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. GroupStatesMode::Blocked(blk_size) => { let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { 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 4963cc8a9634..f91c44be08af 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -356,7 +356,7 @@ impl NullState { } } -/// Similar as the [NullState] but supported the blocked version accumulator +/// Similar as the [NullState] but designed for blocked version accumulator #[derive(Debug)] pub struct BlockedNullState { /// Have we seen any non-filtered input values for `group_index`? @@ -717,7 +717,7 @@ fn initialize_builder( builder } -/// Similar as the [initialize_builder] but supported the blocked version accumulator +/// Similar as the [initialize_builder] but designed for the blocked version accumulator fn ensure_enough_room_for_nulls( builder_blocks: &mut Blocks, mode: GroupStatesMode, @@ -741,7 +741,7 @@ fn ensure_enough_room_for_nulls( builder.append_n(new_groups, default_value); } } - // It blocked mode, we ensure the blks are enough first, + // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. GroupStatesMode::Blocked(blk_size) => { let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 6c727b327afd..4e1b8c6ea0d0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -59,6 +59,7 @@ pub trait GroupValues: Send { } /// Switch the group values to flat or blocked mode. + /// You can see detail about the mode on [GroupStatesMode]. /// /// After switching mode, all data in previous mode will be cleared. fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { From 3b63eaa9a050644b5e6c1226f8a4f69eea470d7e Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 02:41:39 +0800 Subject: [PATCH 040/107] eliminate the unnecessary `mode check + current_mut`. --- .../groups_accumulator/accumulate.rs | 288 +++++++++--------- .../aggregate/groups_accumulator/prim_op.rs | 53 ++-- datafusion/functions-aggregate/src/average.rs | 184 ++++++----- datafusion/functions-aggregate/src/count.rs | 171 ++++++----- 4 files changed, 373 insertions(+), 323 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 f91c44be08af..ac3e8fe5f09a 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -401,13 +401,16 @@ impl BlockedNullState { values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - mut value_fn: F, + value_fn: F, ) where T: ArrowPrimitiveType + Send, F: FnMut(usize, T::Native) + Send, { - let data: &[T::Native] = values.values(); - assert_eq!(data.len(), group_indices.len()); + if total_num_groups == 0 { + return; + } + + assert_eq!(values.values().len(), group_indices.len()); // ensure the seen_values is big enough (start everything at // "not seen" valid) @@ -417,158 +420,33 @@ impl BlockedNullState { total_num_groups, false, ); - let seen_values_blocks = &mut self.seen_values_blocks; - - match (values.null_count() > 0, opt_filter) { - // no nulls, no filter, - (false, None) => { - let iter = group_indices.iter().zip(data.iter()); - for (&group_index, &new_value) in iter { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .current_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - // dbg!(seen_values_blocks.len()); - // dbg!(blocked_index.block_id); - // dbg!(blk_size); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value); - } - } - // nulls, no filter - (true, None) => { - let nulls = values.nulls().unwrap(); - // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum - // iterate over in chunks of 64 bits for more efficient null checking - let group_indices_chunks = group_indices.chunks_exact(64); - let data_chunks = data.chunks_exact(64); - let bit_chunks = nulls.inner().bit_chunks(); - let group_indices_remainder = group_indices_chunks.remainder(); - let data_remainder = data_chunks.remainder(); - - group_indices_chunks - .zip(data_chunks) - .zip(bit_chunks.iter()) - .for_each(|((group_index_chunk, data_chunk), mask)| { - // index_mask has value 1 << i in the loop - let mut index_mask = 1; - group_index_chunk.iter().zip(data_chunk.iter()).for_each( - |(&group_index, &new_value)| { - // valid bit was set, real value - let is_valid = (mask & index_mask) != 0; - if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .current_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit( - blocked_index.block_offset, - true, - ); - } - } - value_fn(group_index, new_value); - } - index_mask <<= 1; - }, - ) - }); + let seen_values_blocks = &mut self.seen_values_blocks; - // handle any remaining bits (after the initial 64) - let remainder_bits = bit_chunks.remainder_bits(); - group_indices_remainder - .iter() - .zip(data_remainder.iter()) - .enumerate() - .for_each(|(i, (&group_index, &new_value))| { - let is_valid = remainder_bits & (1 << i) != 0; - if is_valid { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .current_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value); - } - }); - } - // no nulls, but a filter - (false, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than a single - // iterator. TODO file a ticket - group_indices - .iter() - .zip(data.iter()) - .zip(filter.iter()) - .for_each(|((&group_index, &new_value), filter_value)| { - if let Some(true) = filter_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .current_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value); - } - }) - } - // both null values and filters - (true, Some(filter)) => { - assert_eq!(filter.len(), group_indices.len()); - // The performance with a filter could be improved by - // iterating over the filter in chunks, rather than using - // iterators. TODO file a ticket - filter - .iter() - .zip(group_indices.iter()) - .zip(values.iter()) - .for_each(|((filter_value, &group_index), new_value)| { - if let Some(true) = filter_value { - if let Some(new_value) = new_value { - match self.mode { - GroupStatesMode::Flat => seen_values_blocks - .current_mut() - .unwrap() - .set_bit(group_index, true), - GroupStatesMode::Blocked(_) => { - let blocked_index = - BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - } - } - value_fn(group_index, new_value) - } - } - }) + match self.mode { + GroupStatesMode::Flat => { + let block = seen_values_blocks.current_mut().unwrap(); + accumulate_internal( + group_indices, + values, + opt_filter, + value_fn, + |group_index| { + block.set_bit(group_index, true); + }, + ) } + GroupStatesMode::Blocked(_) => accumulate_internal( + group_indices, + values, + opt_filter, + value_fn, + |group_index| { + let blocked_index = BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + }, + ), } } @@ -701,6 +579,112 @@ pub fn accumulate_indices( } } +fn accumulate_internal( + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + mut value_fn: F1, + mut set_valid_fn: F2, +) where + T: ArrowPrimitiveType + Send, + F1: FnMut(usize, T::Native) + Send, + F2: FnMut(usize) + Send, +{ + let data: &[T::Native] = values.values(); + + match (values.null_count() > 0, opt_filter) { + // no nulls, no filter, + (false, None) => { + let iter = group_indices.iter().zip(data.iter()); + for (&group_index, &new_value) in iter { + set_valid_fn(group_index); + value_fn(group_index, new_value); + } + } + // nulls, no filter + (true, None) => { + let nulls = values.nulls().unwrap(); + // This is based on (ahem, COPY/PASTE) arrow::compute::aggregate::sum + // iterate over in chunks of 64 bits for more efficient null checking + let group_indices_chunks = group_indices.chunks_exact(64); + let data_chunks = data.chunks_exact(64); + let bit_chunks = nulls.inner().bit_chunks(); + + let group_indices_remainder = group_indices_chunks.remainder(); + let data_remainder = data_chunks.remainder(); + + group_indices_chunks + .zip(data_chunks) + .zip(bit_chunks.iter()) + .for_each(|((group_index_chunk, data_chunk), mask)| { + // index_mask has value 1 << i in the loop + let mut index_mask = 1; + group_index_chunk.iter().zip(data_chunk.iter()).for_each( + |(&group_index, &new_value)| { + // valid bit was set, real value + let is_valid = (mask & index_mask) != 0; + if is_valid { + set_valid_fn(group_index); + value_fn(group_index, new_value); + } + index_mask <<= 1; + }, + ) + }); + + // handle any remaining bits (after the initial 64) + let remainder_bits = bit_chunks.remainder_bits(); + group_indices_remainder + .iter() + .zip(data_remainder.iter()) + .enumerate() + .for_each(|(i, (&group_index, &new_value))| { + let is_valid = remainder_bits & (1 << i) != 0; + if is_valid { + set_valid_fn(group_index); + value_fn(group_index, new_value); + } + }); + } + // no nulls, but a filter + (false, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than a single + // iterator. TODO file a ticket + group_indices + .iter() + .zip(data.iter()) + .zip(filter.iter()) + .for_each(|((&group_index, &new_value), filter_value)| { + if let Some(true) = filter_value { + set_valid_fn(group_index); + value_fn(group_index, new_value); + } + }) + } + // both null values and filters + (true, Some(filter)) => { + assert_eq!(filter.len(), group_indices.len()); + // The performance with a filter could be improved by + // iterating over the filter in chunks, rather than using + // iterators. TODO file a ticket + filter + .iter() + .zip(group_indices.iter()) + .zip(values.iter()) + .for_each(|((filter_value, &group_index), new_value)| { + if let Some(true) = filter_value { + if let Some(new_value) = new_value { + set_valid_fn(group_index); + value_fn(group_index, new_value) + } + } + }) + } + } +} + /// Ensures that `builder` contains a `BooleanBufferBuilder with at /// least `total_num_groups`. /// 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 954f16c1c354..c7185069e37a 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 @@ -98,6 +98,10 @@ where opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { + if total_num_groups == 0 { + return Ok(()); + } + assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); @@ -110,28 +114,33 @@ where ); // NullState dispatches / handles tracking nulls and groups that saw no values - self.null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, new_value| { - let value = match self.mode { - GroupStatesMode::Flat => self - .values_blocks - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.values_blocks[blocked_index.block_id] - [blocked_index.block_offset] - } - }; - (self.prim_fn)(value, new_value); - }, - ); + match self.mode { + GroupStatesMode::Flat => { + let block = self.values_blocks.current_mut().unwrap(); + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let value = block.get_mut(group_index).unwrap(); + (self.prim_fn)(value, new_value); + }, + ); + } + GroupStatesMode::Blocked(_) => self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let blocked_index = BlockedGroupIndex::new(group_index); + let value = &mut self.values_blocks[blocked_index.block_id] + [blocked_index.block_offset]; + (self.prim_fn)(value, new_value); + }, + ), + } Ok(()) } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index c32dfbcdbbcd..76b8d6ca7945 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -449,6 +449,10 @@ where opt_filter: Option<&array::BooleanArray>, total_num_groups: usize, ) -> Result<()> { + if total_num_groups == 0 { + return Ok(()); + } + assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); @@ -461,43 +465,44 @@ where T::default_value(), ); - self.null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, new_value| { - let sum = match self.mode { - GroupStatesMode::Flat => self - .sums - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.sums[blocked_index.block_id][blocked_index.block_offset] - } - }; - - let count = match self.mode { - GroupStatesMode::Flat => self - .counts - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { + match self.mode { + GroupStatesMode::Flat => { + let sum_block = self.sums.current_mut().unwrap(); + let count_block = self.counts.current_mut().unwrap(); + + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let sum = sum_block.get_mut(group_index).unwrap(); + let count = count_block.get_mut(group_index).unwrap(); + + *sum = sum.add_wrapping(new_value); + *count += 1; + }, + ); + } + GroupStatesMode::Blocked(_) => { + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset] - } - }; - - *sum = sum.add_wrapping(new_value); - *count += 1; - }, - ); + let sum = &mut self.sums[blocked_index.block_id] + [blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + + *sum = sum.add_wrapping(new_value); + *count += 1; + }, + ); + } + } Ok(()) } @@ -563,64 +568,81 @@ where opt_filter: Option<&array::BooleanArray>, total_num_groups: usize, ) -> Result<()> { + if total_num_groups == 0 { + return Ok(()); + } + assert_eq!(values.len(), 2, "two arguments to merge_batch"); // first batch is counts, second is partial sums let partial_counts = values[0].as_primitive::(); let partial_sums = values[1].as_primitive::(); - // update counts with partial counts ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - self.null_state.accumulate( - group_indices, - partial_counts, - opt_filter, - total_num_groups, - |group_index, partial_count| { - let count = match self.mode { - GroupStatesMode::Flat => self - .counts - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset] - } - }; - *count += partial_count; - }, - ); - - // update sums ensure_enough_room_for_values( &mut self.sums, self.mode, total_num_groups, T::default_value(), ); - self.null_state.accumulate( - group_indices, - partial_sums, - opt_filter, - total_num_groups, - |group_index, new_value: ::Native| { - let sum = match self.mode { - GroupStatesMode::Flat => self - .sums - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { + + match self.mode { + GroupStatesMode::Flat => { + // update counts with partial counts + let count_block = self.counts.current_mut().unwrap(); + self.null_state.accumulate( + group_indices, + partial_counts, + opt_filter, + total_num_groups, + |group_index, partial_count| { + let count = count_block.get_mut(group_index).unwrap(); + *count += partial_count; + }, + ); + + // update sums + let sum_block = self.sums.current_mut().unwrap(); + self.null_state.accumulate( + group_indices, + partial_sums, + opt_filter, + total_num_groups, + |group_index, new_value: ::Native| { + let sum = sum_block.get_mut(group_index).unwrap(); + *sum = sum.add_wrapping(new_value); + }, + ); + } + GroupStatesMode::Blocked(_) => { + // update counts with partial counts + self.null_state.accumulate( + group_indices, + partial_counts, + opt_filter, + total_num_groups, + |group_index, partial_count| { let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.sums[blocked_index.block_id][blocked_index.block_offset] - } - }; - *sum = sum.add_wrapping(new_value); - }, - ); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + *count += partial_count; + }, + ); + + // update sums + self.null_state.accumulate( + group_indices, + partial_sums, + opt_filter, + total_num_groups, + |group_index, new_value: ::Native| { + let blocked_index = BlockedGroupIndex::new(group_index); + let sum = &mut self.sums[blocked_index.block_id] + [blocked_index.block_offset]; + *sum = sum.add_wrapping(new_value); + }, + ); + } + } Ok(()) } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 6e195249e77f..3f84f93180f6 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -384,6 +384,10 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { + if total_num_groups == 0 { + return Ok(()); + } + assert_eq!(values.len(), 1, "single argument to update_batch"); let values = &values[0]; @@ -391,29 +395,23 @@ impl GroupsAccumulator for CountGroupsAccumulator { // filtered value ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - let count = match self.mode { - GroupStatesMode::Flat => self - .counts - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset] - } - }; - *count += 1; - }, - ); - - Ok(()) + match self.mode { + GroupStatesMode::Flat => { + let block = self.counts.current_mut().unwrap(); + update_batch_internal(values, group_indices, opt_filter, |group_index| { + let count = block.get_mut(group_index).unwrap(); + *count += 1; + }) + } + GroupStatesMode::Blocked(_) => { + update_batch_internal(values, group_indices, opt_filter, |group_index| { + let blocked_index = BlockedGroupIndex::new(group_index); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + *count += 1; + }) + } + } } fn merge_batch( @@ -423,61 +421,41 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter: Option<&BooleanArray>, total_num_groups: usize, ) -> Result<()> { - assert_eq!(values.len(), 1, "one argument to merge_batch"); - // first batch is counts, second is partial sums - let partial_counts = values[0].as_primitive::(); + if total_num_groups == 0 { + return Ok(()); + } - // intermediate counts are always created as non null - assert_eq!(partial_counts.null_count(), 0); - let partial_counts = partial_counts.values(); + assert_eq!(values.len(), 1, "one argument to merge_batch"); + let values = &values[0]; // Adds the counts with the partial counts ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - match opt_filter { - Some(filter) => filter - .iter() - .zip(group_indices.iter()) - .zip(partial_counts.iter()) - .for_each(|((filter_value, &group_index), partial_count)| { - if let Some(true) = filter_value { - let count = match self.mode { - GroupStatesMode::Flat => self - .counts - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset] - } - }; + match self.mode { + GroupStatesMode::Flat => { + let block = self.counts.current_mut().unwrap(); + merge_batch_internal( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let count = block.get_mut(group_index).unwrap(); *count += partial_count; - } - }), - None => group_indices.iter().zip(partial_counts.iter()).for_each( - |(&group_index, partial_count)| { - let count = match self.mode { - GroupStatesMode::Flat => self - .counts - .current_mut() - .unwrap() - .get_mut(group_index) - .unwrap(), - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(group_index); - &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset] - } - }; + }, + ) + } + GroupStatesMode::Blocked(_) => merge_batch_internal( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let blocked_index = BlockedGroupIndex::new(group_index); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; *count += partial_count; }, ), } - - Ok(()) } fn evaluate(&mut self, emit_to: EmitTo) -> Result { @@ -600,6 +578,63 @@ fn null_count_for_multiple_cols(values: &[ArrayRef]) -> usize { } } +fn update_batch_internal( + values: &ArrayRef, + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + mut update_group_fn: F, +) -> Result<()> +where + F: FnMut(usize) + Send, +{ + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + update_group_fn(group_index); + }, + ); + + Ok(()) +} + +fn merge_batch_internal( + values: &ArrayRef, + group_indices: &[usize], + opt_filter: Option<&BooleanArray>, + mut update_group_fn: F, +) -> Result<()> +where + F: FnMut(usize, i64), +{ + // first batch is counts, second is partial sums + let partial_counts = values.as_primitive::(); + + // intermediate counts are always created as non null + assert_eq!(partial_counts.null_count(), 0); + let partial_counts = partial_counts.values(); + + match opt_filter { + Some(filter) => filter + .iter() + .zip(group_indices.iter()) + .zip(partial_counts.iter()) + .for_each(|((filter_value, &group_index), &partial_count)| { + if let Some(true) = filter_value { + update_group_fn(group_index, partial_count); + } + }), + None => group_indices.iter().zip(partial_counts.iter()).for_each( + |(&group_index, &partial_count)| { + update_group_fn(group_index, partial_count); + }, + ), + } + + Ok(()) +} + /// General purpose distinct accumulator that works for any DataType by using /// [`ScalarValue`]. /// From 5602dede684f8763812fe6beed608f178c48ee31 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 10:30:53 +0800 Subject: [PATCH 041/107] use index to replace get + unwrap. --- .../src/aggregate/groups_accumulator.rs | 4 +--- .../src/aggregate/groups_accumulator/accumulate.rs | 11 +++-------- .../src/aggregate/groups_accumulator/prim_op.rs | 2 +- datafusion/functions-aggregate/src/average.rs | 8 ++++---- datafusion/functions-aggregate/src/count.rs | 2 +- 5 files changed, 10 insertions(+), 17 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 507e40d12302..4715350940bd 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -431,9 +431,7 @@ pub fn ensure_enough_room_for_values( total_num_groups: usize, default_value: T, ) { - if total_num_groups == 0 { - return; - } + debug_assert!(total_num_groups > 0); match mode { // It flat mode, we just a single builder, and grow it constantly. 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 ac3e8fe5f09a..8e5a47b9fd35 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -406,11 +406,8 @@ impl BlockedNullState { T: ArrowPrimitiveType + Send, F: FnMut(usize, T::Native) + Send, { - if total_num_groups == 0 { - return; - } - - assert_eq!(values.values().len(), group_indices.len()); + debug_assert!(total_num_groups > 0); + debug_assert_eq!(values.values().len(), group_indices.len()); // ensure the seen_values is big enough (start everything at // "not seen" valid) @@ -708,9 +705,7 @@ fn ensure_enough_room_for_nulls( total_num_groups: usize, default_value: bool, ) { - if total_num_groups == 0 { - return; - } + debug_assert!(total_num_groups > 0); match mode { // It flat mode, we just a single builder, and grow it constantly. 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 c7185069e37a..1e7704c9e84d 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 @@ -123,7 +123,7 @@ where opt_filter, total_num_groups, |group_index, new_value| { - let value = block.get_mut(group_index).unwrap(); + let value = &mut block[group_index]; (self.prim_fn)(value, new_value); }, ); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 76b8d6ca7945..3def1bafdd36 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -476,8 +476,8 @@ where opt_filter, total_num_groups, |group_index, new_value| { - let sum = sum_block.get_mut(group_index).unwrap(); - let count = count_block.get_mut(group_index).unwrap(); + let sum = &mut sum_block[group_index]; + let count = &mut count_block[group_index]; *sum = sum.add_wrapping(new_value); *count += 1; @@ -595,7 +595,7 @@ where opt_filter, total_num_groups, |group_index, partial_count| { - let count = count_block.get_mut(group_index).unwrap(); + let count = &mut count_block[group_index]; *count += partial_count; }, ); @@ -608,7 +608,7 @@ where opt_filter, total_num_groups, |group_index, new_value: ::Native| { - let sum = sum_block.get_mut(group_index).unwrap(); + let sum = &mut sum_block[group_index]; *sum = sum.add_wrapping(new_value); }, ); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 3f84f93180f6..5585afdec80c 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -439,7 +439,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { group_indices, opt_filter, |group_index, partial_count| { - let count = block.get_mut(group_index).unwrap(); + let count = &mut block[group_index]; *count += partial_count; }, ) From 8da78061fdcfa66e9ac64ce3afe0b0a0b12eb3ba Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 21:09:58 +0800 Subject: [PATCH 042/107] add comments for some interanl functions. --- datafusion/functions-aggregate/src/count.rs | 62 ++++++++++----------- 1 file changed, 28 insertions(+), 34 deletions(-) diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 5585afdec80c..f4a12a6bb4d8 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -398,20 +398,32 @@ impl GroupsAccumulator for CountGroupsAccumulator { match self.mode { GroupStatesMode::Flat => { let block = self.counts.current_mut().unwrap(); - update_batch_internal(values, group_indices, opt_filter, |group_index| { - let count = block.get_mut(group_index).unwrap(); - *count += 1; - }) + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + let count = block.get_mut(group_index).unwrap(); + *count += 1; + }, + ); } GroupStatesMode::Blocked(_) => { - update_batch_internal(values, group_indices, opt_filter, |group_index| { - let blocked_index = BlockedGroupIndex::new(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; - *count += 1; - }) + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + let blocked_index = BlockedGroupIndex::new(group_index); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + *count += 1; + }, + ); } } + + Ok(()) } fn merge_batch( @@ -434,7 +446,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { match self.mode { GroupStatesMode::Flat => { let block = self.counts.current_mut().unwrap(); - merge_batch_internal( + do_count_merge_batch( values, group_indices, opt_filter, @@ -444,7 +456,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { }, ) } - GroupStatesMode::Blocked(_) => merge_batch_internal( + GroupStatesMode::Blocked(_) => do_count_merge_batch( values, group_indices, opt_filter, @@ -578,28 +590,10 @@ fn null_count_for_multiple_cols(values: &[ArrayRef]) -> usize { } } -fn update_batch_internal( - values: &ArrayRef, - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - mut update_group_fn: F, -) -> Result<()> -where - F: FnMut(usize) + Send, -{ - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - update_group_fn(group_index); - }, - ); - - Ok(()) -} - -fn merge_batch_internal( +/// The intermediate states merging function in count accumulator. +/// It can support blocked and flat mode count accumulator through +/// different `update_group_fn`s. +fn do_count_merge_batch( values: &ArrayRef, group_indices: &[usize], opt_filter: Option<&BooleanArray>, From e8ce09b9f1ef755e8301be11bf46e5a66fe3c47b Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 21:59:33 +0800 Subject: [PATCH 043/107] remove more unnecessary mode checks. --- .../src/aggregate/groups_accumulator.rs | 154 ++++++------- .../groups_accumulator/accumulate.rs | 204 ++++++++++-------- .../aggregate/groups_accumulator/prim_op.rs | 56 +++-- datafusion/functions-aggregate/src/average.rs | 81 ++++--- datafusion/functions-aggregate/src/count.rs | 61 ++++-- 5 files changed, 319 insertions(+), 237 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 4715350940bd..3100c5958381 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -34,12 +34,10 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; +use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::{ Blocks, EmitTo, GroupsAccumulator, VecBlocks, }; -use datafusion_expr_common::{ - accumulator::Accumulator, groups_accumulator::GroupStatesMode, -}; /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] /// @@ -425,87 +423,99 @@ pub(crate) fn slice_and_maybe_filter( /// After expanding: /// values: [x, x, x], [x, x, x], [default, default, default] /// -pub fn ensure_enough_room_for_values( +pub fn ensure_enough_room_for_flat_values( values: &mut VecBlocks, - mode: GroupStatesMode, total_num_groups: usize, default_value: T, ) { debug_assert!(total_num_groups > 0); - match mode { - // It flat mode, we just a single builder, and grow it constantly. - GroupStatesMode::Flat => { - if values.num_blocks() == 0 { - values.push_block(Vec::new()); - } + // It flat mode, we just a single builder, and grow it constantly. + if values.num_blocks() == 0 { + values.push_block(Vec::new()); + } - values - .current_mut() - .unwrap() - .resize(total_num_groups, default_value); - } - // In blocked mode, we ensure the blks are enough first, - // and then ensure slots in blks are enough. - GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { - let cur_blk_idx = values.num_blocks() - 1; - let exist_slots = (values.num_blocks() - 1) * blk_size - + values.current().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; - } + values + .current_mut() + .unwrap() + .resize(total_num_groups, default_value); +} - // Ensure blks are enough. - let exist_blks = values.num_blocks(); - let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - if new_blks > 0 { - for _ in 0..new_blks { - values.push_block(Vec::with_capacity(blk_size)); - } - } +/// Expend blocked values to a big enough size for holding `total_num_groups` groups. +/// +/// For example, +/// +/// before expanding: +/// values: [x, x, x], [x, x, x] (blocks=2, block_size=3) +/// total_num_groups: 8 +/// +/// After expanding: +/// values: [x, x, x], [x, x, x], [default, default, default] +/// +pub fn ensure_enough_room_for_blocked_values( + values: &mut VecBlocks, + total_num_groups: usize, + block_size: usize, + default_value: T, +) { + debug_assert!(total_num_groups > 0); - // Ensure slots are enough. - let mut new_slots = total_num_groups - exist_slots; + // In blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { + let cur_blk_idx = values.num_blocks() - 1; + let exist_slots = + (values.num_blocks() - 1) * block_size + values.current().unwrap().len(); - // Expand current blk. - let cur_blk_rest_slots = blk_size - values[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - // We just need to expand current blocks. - values[cur_blk_idx] - .extend(iter::repeat(default_value.clone()).take(new_slots)); - return; - } + (cur_blk_idx, exist_slots) + } else { + (0, 0) + }; - // Expand current blk to full, and expand next blks - values[cur_blk_idx] - .extend(iter::repeat(default_value.clone()).take(cur_blk_rest_slots)); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; - - // Expand whole blks - let expand_blks = new_slots / blk_size; - for _ in 0..expand_blks { - values[cur_blk_idx] - .extend(iter::repeat(default_value.clone()).take(blk_size)); - cur_blk_idx += 1; - } + // No new groups, don't need to expand, just return. + if exist_slots >= total_num_groups { + return; + } - // Expand the last blk if needed - let last_expand_slots = new_slots % blk_size; - if last_expand_slots > 0 { - values - .current_mut() - .unwrap() - .extend(iter::repeat(default_value.clone()).take(last_expand_slots)); - } + // Ensure blks are enough. + let exist_blks = values.num_blocks(); + let new_blks = (total_num_groups + block_size - 1) / block_size - exist_blks; + if new_blks > 0 { + for _ in 0..new_blks { + values.push_block(Vec::with_capacity(block_size)); } } + + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; + + // Expand current blk. + let cur_blk_rest_slots = block_size - values[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + // We just need to expand current blocks. + values[cur_blk_idx].extend(iter::repeat(default_value.clone()).take(new_slots)); + return; + } + + // Expand current blk to full, and expand next blks + values[cur_blk_idx] + .extend(iter::repeat(default_value.clone()).take(cur_blk_rest_slots)); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + + // Expand whole blks + let expand_blks = new_slots / block_size; + for _ in 0..expand_blks { + values[cur_blk_idx].extend(iter::repeat(default_value.clone()).take(block_size)); + cur_blk_idx += 1; + } + + // Expand the last blk if needed + let last_expand_slots = new_slots % block_size; + if last_expand_slots > 0 { + values + .current_mut() + .unwrap() + .extend(iter::repeat(default_value.clone()).take(last_expand_slots)); + } } 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 8e5a47b9fd35..70c257742637 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -395,7 +395,7 @@ impl BlockedNullState { } /// Similar as [NullState::accumulate] but supported the blocked version accumulator - pub fn accumulate( + pub fn accumulate_for_flat( &mut self, group_indices: &[usize], values: &PrimitiveArray, @@ -411,40 +411,50 @@ impl BlockedNullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - ensure_enough_room_for_nulls( + ensure_enough_room_for_nulls_for_flat( &mut self.seen_values_blocks, - self.mode, total_num_groups, false, ); let seen_values_blocks = &mut self.seen_values_blocks; + let block = seen_values_blocks.current_mut().unwrap(); + do_accumulate(group_indices, values, opt_filter, value_fn, |group_index| { + block.set_bit(group_index, true); + }); + } - match self.mode { - GroupStatesMode::Flat => { - let block = seen_values_blocks.current_mut().unwrap(); - accumulate_internal( - group_indices, - values, - opt_filter, - value_fn, - |group_index| { - block.set_bit(group_index, true); - }, - ) - } - GroupStatesMode::Blocked(_) => accumulate_internal( - group_indices, - values, - opt_filter, - value_fn, - |group_index| { - let blocked_index = BlockedGroupIndex::new(group_index); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - }, - ), - } + /// Similar as [NullState::accumulate] but supported the blocked version accumulator + pub fn accumulate_for_blocked( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + block_size: usize, + value_fn: F, + ) where + T: ArrowPrimitiveType + Send, + F: FnMut(usize, T::Native) + Send, + { + debug_assert!(total_num_groups > 0); + debug_assert_eq!(values.values().len(), group_indices.len()); + + // ensure the seen_values is big enough (start everything at + // "not seen" valid) + ensure_enough_room_for_nulls_for_blocked( + &mut self.seen_values_blocks, + total_num_groups, + block_size, + false, + ); + + let seen_values_blocks = &mut self.seen_values_blocks; + do_accumulate(group_indices, values, opt_filter, value_fn, |group_index| { + let blocked_index = BlockedGroupIndex::new(group_index); + seen_values_blocks[blocked_index.block_id] + .set_bit(blocked_index.block_offset, true); + }); } /// Similar as [NullState::build] but support the blocked version accumulator @@ -576,7 +586,7 @@ pub fn accumulate_indices( } } -fn accumulate_internal( +fn do_accumulate( group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, @@ -699,85 +709,89 @@ fn initialize_builder( } /// Similar as the [initialize_builder] but designed for the blocked version accumulator -fn ensure_enough_room_for_nulls( +fn ensure_enough_room_for_nulls_for_flat( builder_blocks: &mut Blocks, - mode: GroupStatesMode, total_num_groups: usize, default_value: bool, ) { debug_assert!(total_num_groups > 0); - match mode { - // It flat mode, we just a single builder, and grow it constantly. - GroupStatesMode::Flat => { - if builder_blocks.num_blocks() == 0 { - builder_blocks.push_block(BooleanBufferBuilder::new(0)); - } + // It flat mode, we just a single builder, and grow it constantly. + if builder_blocks.num_blocks() == 0 { + builder_blocks.push_block(BooleanBufferBuilder::new(0)); + } - let builder = builder_blocks.current_mut().unwrap(); - if builder.len() < total_num_groups { - let new_groups = total_num_groups - builder.len(); - builder.append_n(new_groups, default_value); - } - } - // In blocked mode, we ensure the blks are enough first, - // and then ensure slots in blks are enough. - GroupStatesMode::Blocked(blk_size) => { - let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { - let cur_blk_idx = builder_blocks.num_blocks() - 1; - let exist_slots = (builder_blocks.num_blocks() - 1) * blk_size - + builder_blocks.current().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; - } + let builder = builder_blocks.current_mut().unwrap(); + if builder.len() < total_num_groups { + let new_groups = total_num_groups - builder.len(); + builder.append_n(new_groups, default_value); + } +} - // Ensure blks are enough - let exist_blks = builder_blocks.num_blocks(); - let new_blks = (total_num_groups + blk_size - 1) / blk_size - exist_blks; - if new_blks > 0 { - for _ in 0..new_blks { - builder_blocks.push_block(BooleanBufferBuilder::new(blk_size)); - } - } +/// Similar as the [initialize_builder] but designed for the blocked version accumulator +fn ensure_enough_room_for_nulls_for_blocked( + builder_blocks: &mut Blocks, + total_num_groups: usize, + block_size: usize, + default_value: bool, +) { + debug_assert!(total_num_groups > 0); - // Ensure slots are enough. - let mut new_slots = total_num_groups - exist_slots; + // In blocked mode, we ensure the blks are enough first, + // and then ensure slots in blks are enough. + let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { + let cur_blk_idx = builder_blocks.num_blocks() - 1; + let exist_slots = (builder_blocks.num_blocks() - 1) * block_size + + builder_blocks.current().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; + } - // Expand current blk. - let cur_blk_rest_slots = blk_size - builder_blocks[cur_blk_idx].len(); - if cur_blk_rest_slots >= new_slots { - builder_blocks[cur_blk_idx].append_n(new_slots, default_value); - return; - } + // Ensure blks are enough + let exist_blks = builder_blocks.num_blocks(); + let new_blks = (total_num_groups + block_size - 1) / block_size - exist_blks; + if new_blks > 0 { + for _ in 0..new_blks { + builder_blocks.push_block(BooleanBufferBuilder::new(block_size)); + } + } - // Expand current blk to full, and expand next blks - builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); - new_slots -= cur_blk_rest_slots; - cur_blk_idx += 1; + // Ensure slots are enough. + let mut new_slots = total_num_groups - exist_slots; - // Expand blks - let expand_blks = new_slots / blk_size; - for _ in 0..expand_blks { - builder_blocks[cur_blk_idx].append_n(blk_size, default_value); - cur_blk_idx += 1; - } + // Expand current blk. + let cur_blk_rest_slots = block_size - builder_blocks[cur_blk_idx].len(); + if cur_blk_rest_slots >= new_slots { + builder_blocks[cur_blk_idx].append_n(new_slots, default_value); + return; + } - // Expand the last blk. - let last_expand_slots = new_slots % blk_size; - if last_expand_slots > 0 { - builder_blocks - .current_mut() - .unwrap() - .append_n(last_expand_slots, default_value); - } - } + // Expand current blk to full, and expand next blks + builder_blocks[cur_blk_idx].append_n(cur_blk_rest_slots, default_value); + new_slots -= cur_blk_rest_slots; + cur_blk_idx += 1; + + // Expand blks + let expand_blks = new_slots / block_size; + for _ in 0..expand_blks { + builder_blocks[cur_blk_idx].append_n(block_size, default_value); + cur_blk_idx += 1; + } + + // Expand the last blk. + let last_expand_slots = new_slots % block_size; + if last_expand_slots > 0 { + builder_blocks + .current_mut() + .unwrap() + .append_n(last_expand_slots, default_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 1e7704c9e84d..3430c5261302 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 @@ -28,7 +28,9 @@ use datafusion_expr_common::groups_accumulator::{ }; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; -use crate::aggregate::groups_accumulator::{ensure_enough_room_for_values, Blocks}; +use crate::aggregate::groups_accumulator::{ + ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, Blocks, +}; /// An accumulator that implements a single operation over /// [`ArrowPrimitiveType`] where the accumulated state is the same as @@ -105,19 +107,18 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); - // Ensure enough room in values - ensure_enough_room_for_values( - &mut self.values_blocks, - self.mode, - total_num_groups, - self.starting_value, - ); - // NullState dispatches / handles tracking nulls and groups that saw no values match self.mode { GroupStatesMode::Flat => { + // Ensure enough room in values + ensure_enough_room_for_flat_values( + &mut self.values_blocks, + total_num_groups, + self.starting_value, + ); + let block = self.values_blocks.current_mut().unwrap(); - self.null_state.accumulate( + self.null_state.accumulate_for_flat( group_indices, values, opt_filter, @@ -128,18 +129,29 @@ where }, ); } - GroupStatesMode::Blocked(_) => self.null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, new_value| { - let blocked_index = BlockedGroupIndex::new(group_index); - let value = &mut self.values_blocks[blocked_index.block_id] - [blocked_index.block_offset]; - (self.prim_fn)(value, new_value); - }, - ), + GroupStatesMode::Blocked(blk_size) => { + // Ensure enough room in values + ensure_enough_room_for_blocked_values( + &mut self.values_blocks, + total_num_groups, + blk_size, + self.starting_value, + ); + + self.null_state.accumulate_for_blocked( + group_indices, + values, + opt_filter, + total_num_groups, + blk_size, + |group_index, new_value| { + let blocked_index = BlockedGroupIndex::new(group_index); + let value = &mut self.values_blocks[blocked_index.block_id] + [blocked_index.block_offset]; + (self.prim_fn)(value, new_value); + }, + ); + } } Ok(()) diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 3def1bafdd36..c0b7d85d4dc6 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -40,10 +40,12 @@ use datafusion_expr::{ }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::accumulate::BlockedNullState; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls::{ filtered_null_mask, set_nulls, }; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, +}; use datafusion_functions_aggregate_common::utils::DecimalAverager; use log::debug; @@ -456,21 +458,20 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); - // increment counts, update sums - ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - ensure_enough_room_for_values( - &mut self.sums, - self.mode, - total_num_groups, - T::default_value(), - ); - match self.mode { GroupStatesMode::Flat => { + // increment counts, update sums + ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); + ensure_enough_room_for_flat_values( + &mut self.sums, + total_num_groups, + T::default_value(), + ); + let sum_block = self.sums.current_mut().unwrap(); let count_block = self.counts.current_mut().unwrap(); - self.null_state.accumulate( + self.null_state.accumulate_for_flat( group_indices, values, opt_filter, @@ -484,12 +485,26 @@ where }, ); } - GroupStatesMode::Blocked(_) => { - self.null_state.accumulate( + GroupStatesMode::Blocked(blk_size) => { + ensure_enough_room_for_blocked_values( + &mut self.counts, + total_num_groups, + blk_size, + 0, + ); + ensure_enough_room_for_blocked_values( + &mut self.sums, + total_num_groups, + blk_size, + T::default_value(), + ); + + self.null_state.accumulate_for_blocked( group_indices, values, opt_filter, total_num_groups, + blk_size, |group_index, new_value| { let blocked_index = BlockedGroupIndex::new(group_index); let sum = &mut self.sums[blocked_index.block_id] @@ -577,19 +592,18 @@ where let partial_counts = values[0].as_primitive::(); let partial_sums = values[1].as_primitive::(); - ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - ensure_enough_room_for_values( - &mut self.sums, - self.mode, - total_num_groups, - T::default_value(), - ); - match self.mode { GroupStatesMode::Flat => { + ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); + ensure_enough_room_for_flat_values( + &mut self.sums, + total_num_groups, + T::default_value(), + ); + // update counts with partial counts let count_block = self.counts.current_mut().unwrap(); - self.null_state.accumulate( + self.null_state.accumulate_for_flat( group_indices, partial_counts, opt_filter, @@ -602,7 +616,7 @@ where // update sums let sum_block = self.sums.current_mut().unwrap(); - self.null_state.accumulate( + self.null_state.accumulate_for_flat( group_indices, partial_sums, opt_filter, @@ -613,13 +627,27 @@ where }, ); } - GroupStatesMode::Blocked(_) => { + GroupStatesMode::Blocked(blk_size) => { + ensure_enough_room_for_blocked_values( + &mut self.counts, + total_num_groups, + blk_size, + 0, + ); + ensure_enough_room_for_blocked_values( + &mut self.sums, + total_num_groups, + blk_size, + T::default_value(), + ); + // update counts with partial counts - self.null_state.accumulate( + self.null_state.accumulate_for_blocked( group_indices, partial_counts, opt_filter, total_num_groups, + blk_size, |group_index, partial_count| { let blocked_index = BlockedGroupIndex::new(group_index); let count = &mut self.counts[blocked_index.block_id] @@ -629,11 +657,12 @@ where ); // update sums - self.null_state.accumulate( + self.null_state.accumulate_for_blocked( group_indices, partial_sums, opt_filter, total_num_groups, + blk_size, |group_index, new_value: ::Native| { let blocked_index = BlockedGroupIndex::new(group_index); let sum = &mut self.sums[blocked_index.block_id] diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index f4a12a6bb4d8..c1a661077793 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -20,7 +20,9 @@ use datafusion_expr::groups_accumulator::{ BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks, }; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; -use datafusion_functions_aggregate_common::aggregate::groups_accumulator::ensure_enough_room_for_values; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, +}; use std::collections::HashSet; use std::ops::BitAnd; use std::{fmt::Debug, sync::Arc}; @@ -393,10 +395,10 @@ impl GroupsAccumulator for CountGroupsAccumulator { // Add one to each group's counter for each non null, non // filtered value - ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - match self.mode { GroupStatesMode::Flat => { + ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); + let block = self.counts.current_mut().unwrap(); accumulate_indices( group_indices, @@ -408,7 +410,14 @@ impl GroupsAccumulator for CountGroupsAccumulator { }, ); } - GroupStatesMode::Blocked(_) => { + GroupStatesMode::Blocked(blk_size) => { + ensure_enough_room_for_blocked_values( + &mut self.counts, + total_num_groups, + blk_size, + 0, + ); + accumulate_indices( group_indices, values.logical_nulls().as_ref(), @@ -441,10 +450,10 @@ impl GroupsAccumulator for CountGroupsAccumulator { let values = &values[0]; // Adds the counts with the partial counts - ensure_enough_room_for_values(&mut self.counts, self.mode, total_num_groups, 0); - match self.mode { GroupStatesMode::Flat => { + ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); + let block = self.counts.current_mut().unwrap(); do_count_merge_batch( values, @@ -454,20 +463,31 @@ impl GroupsAccumulator for CountGroupsAccumulator { let count = &mut block[group_index]; *count += partial_count; }, - ) + ); + } + GroupStatesMode::Blocked(blk_size) => { + ensure_enough_room_for_blocked_values( + &mut self.counts, + total_num_groups, + blk_size, + 0, + ); + + do_count_merge_batch( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let blocked_index = BlockedGroupIndex::new(group_index); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + *count += partial_count; + }, + ); } - GroupStatesMode::Blocked(_) => do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; - *count += partial_count; - }, - ), } + + Ok(()) } fn evaluate(&mut self, emit_to: EmitTo) -> Result { @@ -598,8 +618,7 @@ fn do_count_merge_batch( group_indices: &[usize], opt_filter: Option<&BooleanArray>, mut update_group_fn: F, -) -> Result<()> -where +) where F: FnMut(usize, i64), { // first batch is counts, second is partial sums @@ -625,8 +644,6 @@ where }, ), } - - Ok(()) } /// General purpose distinct accumulator that works for any DataType by using From 48c7e4f03fabb857a45c359c10ac866c4a24073d Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 22:10:43 +0800 Subject: [PATCH 044/107] fix some comments. --- .../src/aggregate/groups_accumulator.rs | 10 +++------- .../src/aggregate/groups_accumulator/accumulate.rs | 14 +++++++------- datafusion/functions-aggregate/src/average.rs | 8 +++----- 3 files changed, 13 insertions(+), 19 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 3100c5958381..aee35ac340d4 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -412,24 +412,22 @@ pub(crate) fn slice_and_maybe_filter( } } -/// Expend blocked values to a big enough size for holding `total_num_groups` groups. +/// Expend flat values to a big enough size for holding `total_num_groups` groups. /// /// For example, /// /// before expanding: -/// values: [x, x, x], [x, x, x] (blocks=2, block_size=3) +/// values: [x, x, x, x, x, x] /// total_num_groups: 8 /// /// After expanding: -/// values: [x, x, x], [x, x, x], [default, default, default] +/// values: [x, x, x, x, x, x, default, default] /// pub fn ensure_enough_room_for_flat_values( values: &mut VecBlocks, total_num_groups: usize, default_value: T, ) { - debug_assert!(total_num_groups > 0); - // It flat mode, we just a single builder, and grow it constantly. if values.num_blocks() == 0 { values.push_block(Vec::new()); @@ -458,8 +456,6 @@ pub fn ensure_enough_room_for_blocked_values( block_size: usize, default_value: T, ) { - debug_assert!(total_num_groups > 0); - // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { 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 70c257742637..d6f56ca80f73 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -394,7 +394,7 @@ impl BlockedNullState { .sum::() } - /// Similar as [NullState::accumulate] but supported the blocked version accumulator + /// Similar as [NullState::accumulate] pub fn accumulate_for_flat( &mut self, group_indices: &[usize], @@ -411,7 +411,7 @@ impl BlockedNullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - ensure_enough_room_for_nulls_for_flat( + ensure_enough_room_for_flat_nulls( &mut self.seen_values_blocks, total_num_groups, false, @@ -424,7 +424,7 @@ impl BlockedNullState { }); } - /// Similar as [NullState::accumulate] but supported the blocked version accumulator + /// Similar as [NullState::accumulate] but designed for blocked version accumulator pub fn accumulate_for_blocked( &mut self, group_indices: &[usize], @@ -442,7 +442,7 @@ impl BlockedNullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - ensure_enough_room_for_nulls_for_blocked( + ensure_enough_room_for_blocked_nulls( &mut self.seen_values_blocks, total_num_groups, block_size, @@ -708,8 +708,8 @@ fn initialize_builder( builder } -/// Similar as the [initialize_builder] but designed for the blocked version accumulator -fn ensure_enough_room_for_nulls_for_flat( +/// Similar as the [initialize_builder] +fn ensure_enough_room_for_flat_nulls( builder_blocks: &mut Blocks, total_num_groups: usize, default_value: bool, @@ -729,7 +729,7 @@ fn ensure_enough_room_for_nulls_for_flat( } /// Similar as the [initialize_builder] but designed for the blocked version accumulator -fn ensure_enough_room_for_nulls_for_blocked( +fn ensure_enough_room_for_blocked_nulls( builder_blocks: &mut Blocks, total_num_groups: usize, block_size: usize, diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index c0b7d85d4dc6..c00022d1fead 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -458,9 +458,9 @@ where assert_eq!(values.len(), 1, "single argument to update_batch"); let values = values[0].as_primitive::(); + // increment counts, update sums match self.mode { GroupStatesMode::Flat => { - // increment counts, update sums ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); ensure_enough_room_for_flat_values( &mut self.sums, @@ -588,10 +588,12 @@ where } assert_eq!(values.len(), 2, "two arguments to merge_batch"); + // first batch is counts, second is partial sums let partial_counts = values[0].as_primitive::(); let partial_sums = values[1].as_primitive::(); + // update counts with partial counts + update sums match self.mode { GroupStatesMode::Flat => { ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); @@ -601,7 +603,6 @@ where T::default_value(), ); - // update counts with partial counts let count_block = self.counts.current_mut().unwrap(); self.null_state.accumulate_for_flat( group_indices, @@ -614,7 +615,6 @@ where }, ); - // update sums let sum_block = self.sums.current_mut().unwrap(); self.null_state.accumulate_for_flat( group_indices, @@ -641,7 +641,6 @@ where T::default_value(), ); - // update counts with partial counts self.null_state.accumulate_for_blocked( group_indices, partial_counts, @@ -656,7 +655,6 @@ where }, ); - // update sums self.null_state.accumulate_for_blocked( group_indices, partial_sums, From 0bbad3a324e1206db8a2d30e6a61a80d7378f5d7 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 22:17:49 +0800 Subject: [PATCH 045/107] improve comments about `Block`. --- datafusion/expr-common/src/groups_accumulator.rs | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index 32f684580e70..eebdec5cf500 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -170,6 +170,21 @@ impl BlockedGroupIndex { } /// The basic data structure for blocked aggregation intermediate results +/// +/// The reason why not use `VecDeque` directly: +/// +/// `current` and `current_mut` will be called frequently, +/// and if we use `VecDeque` directly, they will be mapped +/// to `back` and `back_mut` in it. +/// +/// `back` and `back_mut` are implemented using indexed operation +/// which need some computation about address that will be a bit +/// more expansive than we keep the latest element in `current`, +/// and just return reference of it directly. +/// +/// This small optimization can bring slight performance improvement +/// in the single block case(e.g. when blocked optimization is disabled). +/// pub struct Blocks { /// The current block, it should be pushed into `previous` /// when next block is pushed From 0ab53dceea346a4f5cd2ac5908ec8aa53c741e91 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 21 Aug 2024 22:42:11 +0800 Subject: [PATCH 046/107] move `Blocks`, `BlockedIndex`, some functions of `Emit` to `datafusion-functions-aggregate-common`. --- datafusion-examples/Cargo.toml | 1 + datafusion-examples/examples/advanced_udaf.rs | 1 + .../expr-common/src/groups_accumulator.rs | 272 ----------------- .../src/aggregate/groups_accumulator.rs | 287 +++++++++++++++++- .../groups_accumulator/accumulate.rs | 10 +- .../aggregate/groups_accumulator/prim_op.rs | 5 +- datafusion/functions-aggregate/src/average.rs | 5 +- datafusion/functions-aggregate/src/count.rs | 5 +- .../src/aggregates/group_values/row.rs | 5 +- 9 files changed, 299 insertions(+), 292 deletions(-) diff --git a/datafusion-examples/Cargo.toml b/datafusion-examples/Cargo.toml index 626c365af21c..0269e7bcef6d 100644 --- a/datafusion-examples/Cargo.toml +++ b/datafusion-examples/Cargo.toml @@ -62,6 +62,7 @@ dashmap = { workspace = true } datafusion = { workspace = true, default-features = true, features = ["avro"] } datafusion-common = { workspace = true, default-features = true } datafusion-expr = { workspace = true } +datafusion-functions-aggregate-common = { workspace = true } datafusion-optimizer = { workspace = true, default-features = true } datafusion-physical-expr = { workspace = true, default-features = true } datafusion-proto = { workspace = true } diff --git a/datafusion-examples/examples/advanced_udaf.rs b/datafusion-examples/examples/advanced_udaf.rs index 1259f90d6449..f107731e5b91 100644 --- a/datafusion-examples/examples/advanced_udaf.rs +++ b/datafusion-examples/examples/advanced_udaf.rs @@ -17,6 +17,7 @@ use arrow_schema::{Field, Schema}; use datafusion::{arrow::datatypes::DataType, logical_expr::Volatility}; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::EmitToExt; use datafusion_physical_expr::NullState; use std::{any::Any, sync::Arc}; diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index eebdec5cf500..b52a4e76786d 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -17,19 +17,8 @@ //! Vectorized [`GroupsAccumulator`] -use std::{ - cmp::min, - collections::VecDeque, - iter, - ops::{Index, IndexMut}, -}; - use arrow::array::{ArrayRef, BooleanArray}; use datafusion_common::{not_impl_err, DataFusionError, Result}; -use std::fmt; - -const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; -const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; /// Describes how many rows should be emitted during grouping. #[derive(Debug, Clone, Copy)] @@ -52,68 +41,6 @@ pub enum EmitTo { NextBlock(bool), } -impl EmitTo { - /// 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 { - match self { - Self::All => { - // Take the entire vector, leave new (empty) vector - std::mem::take(v) - } - Self::First(n) => { - // get end n+1,.. values into t - let mut t = v.split_off(*n); - // leave n+1,.. in v - std::mem::swap(v, &mut t); - t - } - Self::NextBlock(_) => unreachable!( - "can not support blocked emission in take_needed, you should use take_needed_from_blocks" - ), - } - } - - /// Removes the number of rows from `blocks` required to emit, - /// returning a `Vec` with elements taken. - /// - /// The detailed behavior in different emissions: - /// - For Emit::CurrentBlock, the first block will be taken and return. - /// - For Emit::All and Emit::First, it will be only supported in `GroupStatesMode::Flat`, - /// similar as `take_needed`. - pub fn take_needed_from_blocks( - &self, - blocks: &mut VecBlocks, - mode: GroupStatesMode, - ) -> Vec { - match self { - Self::All => { - debug_assert!(matches!(mode, GroupStatesMode::Flat)); - blocks.pop_first_block().unwrap() - } - Self::First(n) => { - debug_assert!(matches!(mode, GroupStatesMode::Flat)); - - let block = blocks.current_mut().unwrap(); - let split_at = min(block.len(), *n); - - // get end n+1,.. values into t - let mut t = block.split_off(split_at); - // leave n+1,.. in v - std::mem::swap(block, &mut t); - t - } - Self::NextBlock(_) => { - debug_assert!(matches!(mode, GroupStatesMode::Blocked(_))); - blocks.pop_first_block().unwrap() - } - } - } -} - /// Mode for `accumulators` and `group values` /// /// Their meanings: @@ -131,205 +58,6 @@ pub enum GroupStatesMode { Blocked(usize), } -/// Blocked style group index used in blocked mode group values and accumulators -/// -/// Parts in index: -/// - High 32 bits represent `block_id` -/// - Low 32 bits represent `block_offset` -#[derive(Debug, Clone, Copy)] -pub struct BlockedGroupIndex { - pub block_id: usize, - pub block_offset: usize, -} - -impl BlockedGroupIndex { - pub fn new(group_index: usize) -> Self { - let block_id = - ((group_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - let block_offset = - ((group_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - - Self { - block_id, - block_offset, - } - } - - pub fn new_from_parts(block_id: usize, block_offset: usize) -> Self { - Self { - block_id, - block_offset, - } - } - - pub fn as_packed_index(&self) -> usize { - ((((self.block_id as u64) << 32) & BLOCKED_INDEX_HIGH_32_BITS_MASK) - | (self.block_offset as u64 & BLOCKED_INDEX_LOW_32_BITS_MASK)) - as usize - } -} - -/// The basic data structure for blocked aggregation intermediate results -/// -/// The reason why not use `VecDeque` directly: -/// -/// `current` and `current_mut` will be called frequently, -/// and if we use `VecDeque` directly, they will be mapped -/// to `back` and `back_mut` in it. -/// -/// `back` and `back_mut` are implemented using indexed operation -/// which need some computation about address that will be a bit -/// more expansive than we keep the latest element in `current`, -/// and just return reference of it directly. -/// -/// This small optimization can bring slight performance improvement -/// in the single block case(e.g. when blocked optimization is disabled). -/// -pub struct Blocks { - /// The current block, it should be pushed into `previous` - /// when next block is pushed - current: Option, - - /// Previous blocks pushed before `current` - previous: VecDeque, -} - -impl Blocks { - pub fn new() -> Self { - Self { - current: None, - previous: VecDeque::new(), - } - } - - pub fn current(&self) -> Option<&T> { - self.current.as_ref() - } - - pub fn current_mut(&mut self) -> Option<&mut T> { - self.current.as_mut() - } - - pub fn push_block(&mut self, block: T) { - // If empty, use the block as initialized current - if self.current.is_none() { - self.current = Some(block); - return; - } - - // Take and push the old current to `previous`, - // use input `block` as the new `current` - let old_cur = std::mem::replace(&mut self.current, Some(block)).unwrap(); - self.previous.push_back(old_cur); - } - - pub fn pop_first_block(&mut self) -> Option { - // If `previous` not empty, pop the first of them - if !self.previous.is_empty() { - return self.previous.pop_front(); - } - - // Otherwise, we pop the current - std::mem::take(&mut self.current) - } - - pub fn num_blocks(&self) -> usize { - if self.current.is_none() { - return 0; - } - - self.previous.len() + 1 - } - - // TODO: maybe impl a specific Iterator rather than use the trait object, - // it can slightly improve performance by eliminating the dynamic dispatch. - pub fn iter(&self) -> Box + '_> { - // If current is None, it means no data, return empty iter - if self.current.is_none() { - return Box::new(iter::empty()); - } - - let cur_iter = iter::once(self.current.as_ref().unwrap()); - - if !self.previous.is_empty() { - let previous_iter = self.previous.iter(); - Box::new(previous_iter.chain(cur_iter)) - } else { - Box::new(cur_iter) - } - } - - // TODO: maybe impl a specific Iterator rather than use the trait object, - // it can slightly improve performance by eliminating the dynamic dispatch. - pub fn iter_mut(&mut self) -> Box + '_> { - // If current is None, it means no data, return empty iter - if self.current.is_none() { - return Box::new(iter::empty()); - } - - let cur_iter = iter::once(self.current.as_mut().unwrap()); - - if !self.previous.is_empty() { - let previous_iter = self.previous.iter_mut(); - Box::new(previous_iter.chain(cur_iter)) - } else { - Box::new(cur_iter) - } - } - - pub fn clear(&mut self) { - *self = Self::new(); - } -} - -impl fmt::Debug for Blocks { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.debug_struct("Blocks") - .field("current", &self.current) - .field("previous", &self.previous) - .finish() - } -} - -impl Index for Blocks { - type Output = T; - - fn index(&self, index: usize) -> &T { - if index < self.previous.len() { - &self.previous[index] - } else { - self.current.as_ref().unwrap() - } - } -} - -impl IndexMut for Blocks { - fn index_mut(&mut self, index: usize) -> &mut T { - if index < self.previous.len() { - &mut self.previous[index] - } else { - self.current.as_mut().unwrap() - } - } -} - -impl Default for Blocks { - fn default() -> Self { - Self::new() - } -} - -pub type VecBlocks = Blocks>; - -impl VecBlocks { - pub fn capacity(&self) -> usize { - let cur_cap = self.current.as_ref().map(|blk| blk.capacity()).unwrap_or(0); - let prev_cap = self.previous.iter().map(|p| p.capacity()).sum::(); - - cur_cap + prev_cap - } -} - /// `GroupAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index aee35ac340d4..7d809f61e828 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -23,7 +23,12 @@ pub mod bool_op; pub mod nulls; pub mod prim_op; -use std::iter; +use std::{ + cmp::min, + collections::VecDeque, + fmt, iter, + ops::{Index, IndexMut}, +}; use arrow::{ array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}, @@ -34,11 +39,14 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; -use datafusion_expr_common::accumulator::Accumulator; -use datafusion_expr_common::groups_accumulator::{ - Blocks, EmitTo, GroupsAccumulator, VecBlocks, +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; +use datafusion_expr_common::{ + accumulator::Accumulator, groups_accumulator::GroupStatesMode, }; +const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; +const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; + /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] /// /// While [`Accumulator`] are simpler to implement and can support @@ -371,6 +379,277 @@ impl VecAllocExt for Vec { } } +pub trait EmitToExt { + /// 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 + fn take_needed(&self, v: &mut Vec) -> Vec; + + /// Removes the number of rows from `blocks` required to emit, + /// returning a `Vec` with elements taken. + /// + /// The detailed behavior in different emissions: + /// - For Emit::CurrentBlock, the first block will be taken and return. + /// - For Emit::All and Emit::First, it will be only supported in `GroupStatesMode::Flat`, + /// similar as `take_needed`. + fn take_needed_from_blocks( + &self, + blocks: &mut VecBlocks, + mode: GroupStatesMode, + ) -> Vec; +} + +impl EmitToExt for EmitTo { + fn take_needed(&self, v: &mut Vec) -> Vec { + match self { + Self::All => { + // Take the entire vector, leave new (empty) vector + std::mem::take(v) + } + Self::First(n) => { + // get end n+1,.. values into t + let mut t = v.split_off(*n); + // leave n+1,.. in v + std::mem::swap(v, &mut t); + t + } + Self::NextBlock(_) => unreachable!( + "can not support blocked emission in take_needed, you should use take_needed_from_blocks" + ), + } + } + + fn take_needed_from_blocks( + &self, + blocks: &mut VecBlocks, + mode: GroupStatesMode, + ) -> Vec { + match self { + Self::All => { + debug_assert!(matches!(mode, GroupStatesMode::Flat)); + blocks.pop_first_block().unwrap() + } + Self::First(n) => { + debug_assert!(matches!(mode, GroupStatesMode::Flat)); + + let block = blocks.current_mut().unwrap(); + let split_at = min(block.len(), *n); + + // get end n+1,.. values into t + let mut t = block.split_off(split_at); + // leave n+1,.. in v + std::mem::swap(block, &mut t); + t + } + Self::NextBlock(_) => { + debug_assert!(matches!(mode, GroupStatesMode::Blocked(_))); + blocks.pop_first_block().unwrap() + } + } + } +} + +/// Blocked style group index used in blocked mode group values and accumulators +/// +/// Parts in index: +/// - High 32 bits represent `block_id` +/// - Low 32 bits represent `block_offset` +#[derive(Debug, Clone, Copy)] +pub struct BlockedGroupIndex { + pub block_id: usize, + pub block_offset: usize, +} + +impl BlockedGroupIndex { + pub fn new(group_index: usize) -> Self { + let block_id = + ((group_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + let block_offset = + ((group_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + + Self { + block_id, + block_offset, + } + } + + pub fn new_from_parts(block_id: usize, block_offset: usize) -> Self { + Self { + block_id, + block_offset, + } + } + + pub fn as_packed_index(&self) -> usize { + ((((self.block_id as u64) << 32) & BLOCKED_INDEX_HIGH_32_BITS_MASK) + | (self.block_offset as u64 & BLOCKED_INDEX_LOW_32_BITS_MASK)) + as usize + } +} + +/// The basic data structure for blocked aggregation intermediate results +/// +/// The reason why not use `VecDeque` directly: +/// +/// `current` and `current_mut` will be called frequently, +/// and if we use `VecDeque` directly, they will be mapped +/// to `back` and `back_mut` in it. +/// +/// `back` and `back_mut` are implemented using indexed operation +/// which need some computation about address that will be a bit +/// more expansive than we keep the latest element in `current`, +/// and just return reference of it directly. +/// +/// This small optimization can bring slight performance improvement +/// in the single block case(e.g. when blocked optimization is disabled). +/// +pub struct Blocks { + /// The current block, it should be pushed into `previous` + /// when next block is pushed + current: Option, + + /// Previous blocks pushed before `current` + previous: VecDeque, +} + +impl Blocks { + pub fn new() -> Self { + Self { + current: None, + previous: VecDeque::new(), + } + } + + pub fn current(&self) -> Option<&T> { + self.current.as_ref() + } + + pub fn current_mut(&mut self) -> Option<&mut T> { + self.current.as_mut() + } + + pub fn push_block(&mut self, block: T) { + // If empty, use the block as initialized current + if self.current.is_none() { + self.current = Some(block); + return; + } + + // Take and push the old current to `previous`, + // use input `block` as the new `current` + let old_cur = std::mem::replace(&mut self.current, Some(block)).unwrap(); + self.previous.push_back(old_cur); + } + + pub fn pop_first_block(&mut self) -> Option { + // If `previous` not empty, pop the first of them + if !self.previous.is_empty() { + return self.previous.pop_front(); + } + + // Otherwise, we pop the current + std::mem::take(&mut self.current) + } + + pub fn num_blocks(&self) -> usize { + if self.current.is_none() { + return 0; + } + + self.previous.len() + 1 + } + + // TODO: maybe impl a specific Iterator rather than use the trait object, + // it can slightly improve performance by eliminating the dynamic dispatch. + pub fn iter(&self) -> Box + '_> { + // If current is None, it means no data, return empty iter + if self.current.is_none() { + return Box::new(iter::empty()); + } + + let cur_iter = iter::once(self.current.as_ref().unwrap()); + + if !self.previous.is_empty() { + let previous_iter = self.previous.iter(); + Box::new(previous_iter.chain(cur_iter)) + } else { + Box::new(cur_iter) + } + } + + // TODO: maybe impl a specific Iterator rather than use the trait object, + // it can slightly improve performance by eliminating the dynamic dispatch. + pub fn iter_mut(&mut self) -> Box + '_> { + // If current is None, it means no data, return empty iter + if self.current.is_none() { + return Box::new(iter::empty()); + } + + let cur_iter = iter::once(self.current.as_mut().unwrap()); + + if !self.previous.is_empty() { + let previous_iter = self.previous.iter_mut(); + Box::new(previous_iter.chain(cur_iter)) + } else { + Box::new(cur_iter) + } + } + + pub fn clear(&mut self) { + *self = Self::new(); + } +} + +impl fmt::Debug for Blocks { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.debug_struct("Blocks") + .field("current", &self.current) + .field("previous", &self.previous) + .finish() + } +} + +impl Index for Blocks { + type Output = T; + + fn index(&self, index: usize) -> &T { + if index < self.previous.len() { + &self.previous[index] + } else { + self.current.as_ref().unwrap() + } + } +} + +impl IndexMut for Blocks { + fn index_mut(&mut self, index: usize) -> &mut T { + if index < self.previous.len() { + &mut self.previous[index] + } else { + self.current.as_mut().unwrap() + } + } +} + +impl Default for Blocks { + fn default() -> Self { + Self::new() + } +} + +pub type VecBlocks = Blocks>; + +impl VecBlocks { + pub fn capacity(&self) -> usize { + let cur_cap = self.current.as_ref().map(|blk| blk.capacity()).unwrap_or(0); + let prev_cap = self.previous.iter().map(|p| p.capacity()).sum::(); + + cur_cap + prev_cap + } +} + fn get_filter_at_indices( opt_filter: Option<&BooleanArray>, indices: &PrimitiveArray, 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 d6f56ca80f73..bfe8e03de10c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -23,9 +23,9 @@ use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_expr_common::groups_accumulator::{ - BlockedGroupIndex, Blocks, EmitTo, GroupStatesMode, -}; +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupStatesMode}; + +use crate::aggregate::groups_accumulator::{BlockedGroupIndex, 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. /// @@ -714,8 +714,6 @@ fn ensure_enough_room_for_flat_nulls( total_num_groups: usize, default_value: bool, ) { - debug_assert!(total_num_groups > 0); - // It flat mode, we just a single builder, and grow it constantly. if builder_blocks.num_blocks() == 0 { builder_blocks.push_block(BooleanBufferBuilder::new(0)); @@ -735,8 +733,6 @@ fn ensure_enough_room_for_blocked_nulls( block_size: usize, default_value: bool, ) { - debug_assert!(total_num_groups > 0); - // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { 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 3430c5261302..8ef8a7267926 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 @@ -24,12 +24,13 @@ use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_expr_common::groups_accumulator::{ - BlockedGroupIndex, EmitTo, GroupStatesMode, GroupsAccumulator, VecBlocks, + EmitTo, GroupStatesMode, GroupsAccumulator, }; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::{ - ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, Blocks, + ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, + BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; /// An accumulator that implements a single operation over diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index c00022d1fead..ab45eea7ca4e 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,9 +29,7 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::groups_accumulator::{ - BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks, -}; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; @@ -45,6 +43,7 @@ use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls: }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, + BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; use datafusion_functions_aggregate_common::utils::DecimalAverager; diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index c1a661077793..c607b442ccf3 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,12 +16,11 @@ // under the License. use ahash::RandomState; -use datafusion_expr::groups_accumulator::{ - BlockedGroupIndex, Blocks, GroupStatesMode, VecBlocks, -}; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, + BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; use std::collections::HashSet; use std::ops::BitAnd; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 4fd3501edfef..684e8412bfe8 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -27,8 +27,11 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; -use datafusion_expr::groups_accumulator::{BlockedGroupIndex, Blocks, GroupStatesMode}; +use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::EmitTo; +use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ + BlockedGroupIndex, Blocks, +}; use hashbrown::raw::RawTable; /// A [`GroupValues`] making use of [`Rows`] From 78c8e82d6e42576eaa1bdee5ab600b42f5783e1f Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 22 Aug 2024 02:36:54 +0800 Subject: [PATCH 047/107] add test for `BlockedNullState`. --- .../groups_accumulator/accumulate.rs | 269 +++++++++++++++++- 1 file changed, 255 insertions(+), 14 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 bfe8e03de10c..2fc535d1074e 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -799,8 +799,238 @@ mod test { use rand::{rngs::ThreadRng, Rng}; use std::collections::HashSet; + trait TestNullState { + 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, T::Native) + Send; + + fn build_bool_buffer(&self) -> BooleanBuffer; + + fn build_null_buffer(&mut self) -> NullBuffer; + } + + // The original `NullState` + impl TestNullState for NullState { + 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, T::Native) + Send, + { + self.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ); + } + + fn build_bool_buffer(&self) -> BooleanBuffer { + self.seen_values.finish_cloned() + } + + fn build_null_buffer(&mut self) -> NullBuffer { + self.build(EmitTo::All) + } + } + + // The new `BlockedNullState` in flat mode + struct BlockedNullStateInFlatMode(BlockedNullState); + + impl BlockedNullStateInFlatMode { + fn new() -> Self { + let null_state = BlockedNullState::new(GroupStatesMode::Flat); + + Self(null_state) + } + } + + impl TestNullState for BlockedNullStateInFlatMode { + 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, T::Native) + Send, + { + self.0.accumulate_for_flat( + group_indices, + values, + opt_filter, + total_num_groups, + value_fn, + ); + } + + fn build_bool_buffer(&self) -> BooleanBuffer { + self.0.seen_values_blocks.current().unwrap().finish_cloned() + } + + fn build_null_buffer(&mut self) -> NullBuffer { + self.0.build(EmitTo::All) + } + } + + // The new `BlockedNullState` in blocked mode + struct BlockedNullStateInBlockedMode { + null_state: BlockedNullState, + block_size: usize, + } + + impl BlockedNullStateInBlockedMode { + fn new() -> Self { + let null_state = BlockedNullState::new(GroupStatesMode::Blocked(4)); + + Self { + null_state, + block_size: 4, + } + } + } + + impl TestNullState for BlockedNullStateInBlockedMode { + 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, T::Native) + Send, + { + self.null_state.accumulate_for_blocked( + group_indices, + values, + opt_filter, + total_num_groups, + self.block_size, + value_fn, + ); + } + + fn build_bool_buffer(&self) -> BooleanBuffer { + let mut ret_builder = BooleanBufferBuilder::new(0); + for blk in self.null_state.seen_values_blocks.iter() { + let buf = blk.finish_cloned(); + for seen in buf.iter() { + ret_builder.append(seen); + } + } + ret_builder.finish() + } + + fn build_null_buffer(&mut self) -> NullBuffer { + let mut init_buffer = NullBuffer::new(BooleanBufferBuilder::new(0).finish()); + loop { + let blk = self.null_state.build(EmitTo::NextBlock(false)); + if blk.is_empty() { + break; + } + + init_buffer = NullBuffer::union(Some(&init_buffer), Some(&blk)).unwrap(); + } + + init_buffer + } + } + + #[derive(Debug, Clone, Copy)] + enum AccumulateTest { + Original, + Flat, + Blocked, + } + + impl AccumulateTest { + fn run( + &self, + group_indices: &[usize], + values: &UInt32Array, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) { + match self { + AccumulateTest::Original => { + Fixture::accumulate_test( + group_indices, + &values, + opt_filter, + total_num_groups, + NullState::new(), + ); + } + AccumulateTest::Flat => { + Fixture::accumulate_test( + group_indices, + &values, + opt_filter, + total_num_groups, + BlockedNullStateInFlatMode::new(), + ); + } + AccumulateTest::Blocked => { + Fixture::accumulate_test( + group_indices, + &values, + opt_filter, + total_num_groups, + BlockedNullStateInBlockedMode::new(), + ); + } + } + } + } + + #[test] + fn accumulate_test_original() { + do_accumulate_test(AccumulateTest::Original); + } + + #[test] + fn accumulate_test_flat() { + do_accumulate_test(AccumulateTest::Flat); + } + + #[test] + fn accumulate_test_blocked() { + do_accumulate_test(AccumulateTest::Blocked); + } + + #[test] + fn accumulate_fuzz_test_original() { + do_accumulate_fuzz_test(AccumulateTest::Original); + } + + #[test] + fn accumulate_fuzz_test_flat() { + do_accumulate_fuzz_test(AccumulateTest::Flat); + } + #[test] - fn accumulate() { + fn accumulate_fuzz_test_blocked() { + do_accumulate_fuzz_test(AccumulateTest::Blocked); + } + + fn do_accumulate_test(accumulate_test: AccumulateTest) { let group_indices = (0..100).collect(); let values = (0..100).map(|i| (i + 1) * 10).collect(); let values_with_nulls = (0..100) @@ -828,15 +1058,15 @@ mod test { values, values_with_nulls, filter, + accumulate_test, } .run() } - #[test] - fn accumulate_fuzz() { + fn do_accumulate_fuzz_test(accumulate_test: AccumulateTest) { let mut rng = rand::thread_rng(); for _ in 0..100 { - Fixture::new_random(&mut rng).run(); + Fixture::new_random(&mut rng, accumulate_test).run(); } } @@ -854,10 +1084,13 @@ mod test { /// filter (defaults to None) filter: BooleanArray, + + /// tested null state for value test + accumulate_test: AccumulateTest, } impl Fixture { - fn new_random(rng: &mut ThreadRng) -> Self { + fn new_random(rng: &mut ThreadRng, accumulate_test: AccumulateTest) -> Self { // Number of input values in a batch let num_values: usize = rng.gen_range(1..200); // number of distinct groups @@ -905,6 +1138,7 @@ mod test { values, values_with_nulls, filter, + accumulate_test, } } @@ -929,10 +1163,15 @@ mod test { let filter = &self.filter; // no null, no filters - Self::accumulate_test(group_indices, &values_array, None, total_num_groups); + self.accumulate_test.run( + group_indices, + &values_array, + None, + total_num_groups, + ); // nulls, no filters - Self::accumulate_test( + self.accumulate_test.run( group_indices, &values_with_nulls_array, None, @@ -940,7 +1179,7 @@ mod test { ); // no nulls, filters - Self::accumulate_test( + self.accumulate_test.run( group_indices, &values_array, Some(filter), @@ -948,7 +1187,7 @@ mod test { ); // nulls, filters - Self::accumulate_test( + self.accumulate_test.run( group_indices, &values_with_nulls_array, Some(filter), @@ -959,17 +1198,19 @@ mod test { /// Calls `NullState::accumulate` and `accumulate_indices` to /// ensure it generates the correct values. /// - fn accumulate_test( + fn accumulate_test( group_indices: &[usize], values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + null_state_for_value_test: SV, ) { Self::accumulate_values_test( group_indices, values, opt_filter, total_num_groups, + null_state_for_value_test, ); Self::accumulate_indices_test(group_indices, values.nulls(), opt_filter); @@ -988,14 +1229,14 @@ mod test { /// This is effectively a different implementation of /// accumulate that we compare with the above implementation - fn accumulate_values_test( + fn accumulate_values_test( group_indices: &[usize], values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, + mut null_state: S, ) { let mut accumulated_values = vec![]; - let mut null_state = NullState::new(); null_state.accumulate( group_indices, @@ -1039,13 +1280,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.finish_cloned(); + let seen_values = null_state.build_bool_buffer(); 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_null_buffer(); assert_eq!(null_buffer, expected_null_buffer); } From f54878fd4f8d2cd3787ce86d72bd54d45ad1610c Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 22 Aug 2024 03:16:51 +0800 Subject: [PATCH 048/107] fix `BlockedNullState`'s unit test. --- .../groups_accumulator/accumulate.rs | 79 ++++++++++++------- 1 file changed, 51 insertions(+), 28 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 2fc535d1074e..fd462140dc92 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -799,42 +799,43 @@ mod test { use rand::{rngs::ThreadRng, Rng}; use std::collections::HashSet; + /// Null state's behaviors needed in accumulate test trait TestNullState { - fn accumulate( + fn accumulate( &mut self, group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - value_fn: F, + accumulated_values: &mut Vec<(usize, T::Native)>, ) where - T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send; + T: ArrowPrimitiveType + Send; fn build_bool_buffer(&self) -> BooleanBuffer; fn build_null_buffer(&mut self) -> NullBuffer; } - // The original `NullState` + /// The original `NullState` impl TestNullState for NullState { - fn accumulate( + fn accumulate( &mut self, group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - value_fn: F, + accumulated_values: &mut Vec<(usize, T::Native)>, ) where T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, { self.accumulate( group_indices, values, opt_filter, total_num_groups, - value_fn, + |group_index, value| { + accumulated_values.push((group_index, value)); + }, ); } @@ -847,7 +848,7 @@ mod test { } } - // The new `BlockedNullState` in flat mode + /// The new `BlockedNullState` in flat mode struct BlockedNullStateInFlatMode(BlockedNullState); impl BlockedNullStateInFlatMode { @@ -859,23 +860,24 @@ mod test { } impl TestNullState for BlockedNullStateInFlatMode { - fn accumulate( + fn accumulate( &mut self, group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - value_fn: F, + accumulated_values: &mut Vec<(usize, T::Native)>, ) where T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, { self.0.accumulate_for_flat( group_indices, values, opt_filter, total_num_groups, - value_fn, + |group_index, value| { + accumulated_values.push((group_index, value)); + }, ); } @@ -888,7 +890,7 @@ mod test { } } - // The new `BlockedNullState` in blocked mode + /// The new `BlockedNullState` in blocked mode struct BlockedNullStateInBlockedMode { null_state: BlockedNullState, block_size: usize, @@ -906,24 +908,39 @@ mod test { } impl TestNullState for BlockedNullStateInBlockedMode { - fn accumulate( + fn accumulate( &mut self, group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - value_fn: F, + accumulated_values: &mut Vec<(usize, T::Native)>, ) where T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, { + // Convert group indices to blocked style + let blocked_indices = group_indices + .iter() + .map(|idx| { + let block_id = *idx / self.block_size; + let block_offset = *idx % self.block_size; + BlockedGroupIndex::new_from_parts(block_id, block_offset) + .as_packed_index() + }) + .collect::>(); + self.null_state.accumulate_for_blocked( - group_indices, + &blocked_indices, values, opt_filter, total_num_groups, self.block_size, - value_fn, + |group_index, value| { + let blocked_index = BlockedGroupIndex::new(group_index); + let flat_index = blocked_index.block_id * self.block_size + + blocked_index.block_offset; + accumulated_values.push((flat_index, value)); + }, ); } @@ -939,20 +956,26 @@ mod test { } fn build_null_buffer(&mut self) -> NullBuffer { - let mut init_buffer = NullBuffer::new(BooleanBufferBuilder::new(0).finish()); + let mut ret_builder = BooleanBufferBuilder::new(0); loop { let blk = self.null_state.build(EmitTo::NextBlock(false)); if blk.is_empty() { break; } - init_buffer = NullBuffer::union(Some(&init_buffer), Some(&blk)).unwrap(); + for seen in blk.iter() { + ret_builder.append(seen); + } } - init_buffer + NullBuffer::new(ret_builder.finish()) } } + /// Accumulate test mode + /// - Original, test the original `NullState` + /// - Flat, test the `BlockedNullState` in flat mode + /// - Blocked, test the `BlockedNullState` in blocked mode #[derive(Debug, Clone, Copy)] enum AccumulateTest { Original, @@ -1198,12 +1221,12 @@ mod test { /// Calls `NullState::accumulate` and `accumulate_indices` to /// ensure it generates the correct values. /// - fn accumulate_test( + fn accumulate_test( group_indices: &[usize], values: &UInt32Array, opt_filter: Option<&BooleanArray>, total_num_groups: usize, - null_state_for_value_test: SV, + null_state_for_value_test: S, ) { Self::accumulate_values_test( group_indices, @@ -1219,6 +1242,8 @@ mod test { 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(); + + // TODO: test the `BlockedNullState` after supporting `accumulate_boolean` in it Self::accumulate_boolean_test( group_indices, &boolean_values, @@ -1243,9 +1268,7 @@ mod test { values, opt_filter, total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); - }, + &mut accumulated_values, ); // Figure out the expected values From 56b0bcf35f10ab05ae43ab87e88d6bd52be3c299 Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 22 Aug 2024 23:56:00 +0800 Subject: [PATCH 049/107] add unit tests for blocks. --- .../src/aggregate/groups_accumulator.rs | 94 +++++++++++++++++++ 1 file changed, 94 insertions(+) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 7d809f61e828..d99499d3a894 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -648,6 +648,13 @@ impl VecBlocks { cur_cap + prev_cap } + + pub fn len(&self) -> usize { + let cur_len = self.current.as_ref().map(|blk| blk.len()).unwrap_or(0); + let prev_len = self.previous.iter().map(|p| p.len()).sum::(); + + cur_len + prev_len + } } fn get_filter_at_indices( @@ -794,3 +801,90 @@ pub fn ensure_enough_room_for_blocked_values( .extend(iter::repeat(default_value.clone()).take(last_expand_slots)); } } + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_ensure_room_for_blocked_values() { + let mut blocks = VecBlocks::new(); + let block_size = 4; + + // 0 total_num_groups, should be no blocks + ensure_enough_room_for_blocked_values(&mut blocks, 0, block_size, 0); + assert_eq!(blocks.num_blocks(), 0); + assert_eq!(blocks.len(), 0); + + // 0 -> 3 total_num_groups, blocks should look like: + // [d, d, d, empty] + ensure_enough_room_for_blocked_values(&mut blocks, 3, block_size, 0); + assert_eq!(blocks.num_blocks(), 1); + assert_eq!(blocks.len(), 3); + + // 3 -> 8 total_num_groups, blocks should look like: + // [d, d, d, d], [d, d, d, d] + ensure_enough_room_for_blocked_values(&mut blocks, 8, block_size, 0); + assert_eq!(blocks.num_blocks(), 2); + assert_eq!(blocks.len(), 8); + + // 8 -> 13 total_num_groups, blocks should look like: + // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] + ensure_enough_room_for_blocked_values(&mut blocks, 13, block_size, 0); + assert_eq!(blocks.num_blocks(), 4); + assert_eq!(blocks.len(), 13); + } + + #[test] + fn test_blocks_ops() { + let mut blocks = VecBlocks::::new(); + + // Test empty blocks + assert!(blocks.current().is_none()); + assert!(blocks.current_mut().is_none()); + assert!(blocks.pop_first_block().is_none()); + assert_eq!(blocks.num_blocks(), 0); + { + let mut iter = blocks.iter(); + assert!(iter.next().is_none()); + } + { + let mut iter_mut = blocks.iter_mut(); + assert!(iter_mut.next().is_none()); + } + + // Test push block + for cnt in 0..100 { + blocks.push_block(Vec::with_capacity(4)); + + assert!(blocks.current().is_some()); + assert!(blocks.current_mut().is_some()); + assert_eq!(blocks.num_blocks(), cnt + 1); + + let block_num = blocks.iter().count(); + assert_eq!(block_num, cnt + 1); + let block_num = blocks.iter_mut().count(); + assert_eq!(block_num, cnt + 1); + } + + // Test pop block + for cnt in 0..100 { + blocks.pop_first_block(); + + let rest_blk_num = 100 - cnt - 1; + assert_eq!(blocks.num_blocks(), rest_blk_num); + if rest_blk_num > 0 { + assert!(blocks.current().is_some()); + assert!(blocks.current_mut().is_some()); + } else { + assert!(blocks.current().is_none()); + assert!(blocks.current_mut().is_none()); + } + + let block_num = blocks.iter().count(); + assert_eq!(block_num, rest_blk_num); + let block_num = blocks.iter_mut().count(); + assert_eq!(block_num, rest_blk_num); + } + } +} From 94af6942ef9dfe8c950e15dcbd2aaf4b78e25d2e Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 23 Aug 2024 00:10:38 +0800 Subject: [PATCH 050/107] add unit test for `ensure_enough_room_for_blocked_nulls`. --- .../groups_accumulator/accumulate.rs | 33 +++++++++++++++++++ 1 file changed, 33 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 fd462140dc92..126b85d17458 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -1471,4 +1471,37 @@ mod test { .collect() } } + + #[test] + fn test_ensure_room_for_blocked_nulls() { + let mut blocks: Blocks = Blocks::new(); + let block_size = 4; + + // 0 total_num_groups, should be no blocks + ensure_enough_room_for_blocked_nulls(&mut blocks, 0, block_size, false); + assert_eq!(blocks.num_blocks(), 0); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 0); + + // 0 -> 3 total_num_groups, blocks should look like: + // [d, d, d, empty] + ensure_enough_room_for_blocked_nulls(&mut blocks, 3, block_size, false); + assert_eq!(blocks.num_blocks(), 1); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 3); + + // 3 -> 8 total_num_groups, blocks should look like: + // [d, d, d, d], [d, d, d, d] + ensure_enough_room_for_blocked_nulls(&mut blocks, 8, block_size, false); + assert_eq!(blocks.num_blocks(), 2); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 8); + + // 8 -> 13 total_num_groups, blocks should look like: + // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] + ensure_enough_room_for_blocked_nulls(&mut blocks, 13, block_size, false); + assert_eq!(blocks.num_blocks(), 4); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 13); + } } From 1064a72385a0fdd9301c41885eafc8075ee237ce Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 23 Aug 2024 00:43:28 +0800 Subject: [PATCH 051/107] test take needed. --- .../src/aggregate/groups_accumulator.rs | 62 ++++++++++++++++++- 1 file changed, 59 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 d99499d3a894..cbb042e05804 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -409,8 +409,9 @@ impl EmitToExt for EmitTo { std::mem::take(v) } Self::First(n) => { + let split_at = min(v.len(), *n); // get end n+1,.. values into t - let mut t = v.split_off(*n); + let mut t = v.split_off(split_at); // leave n+1,.. in v std::mem::swap(v, &mut t); t @@ -429,7 +430,7 @@ impl EmitToExt for EmitTo { match self { Self::All => { debug_assert!(matches!(mode, GroupStatesMode::Flat)); - blocks.pop_first_block().unwrap() + blocks.pop_first_block().unwrap_or(Vec::new()) } Self::First(n) => { debug_assert!(matches!(mode, GroupStatesMode::Flat)); @@ -445,7 +446,7 @@ impl EmitToExt for EmitTo { } Self::NextBlock(_) => { debug_assert!(matches!(mode, GroupStatesMode::Blocked(_))); - blocks.pop_first_block().unwrap() + blocks.pop_first_block().unwrap_or(Vec::new()) } } } @@ -887,4 +888,59 @@ mod tests { assert_eq!(block_num, rest_blk_num); } } + + #[test] + fn test_take_need() { + let values = vec![1, 2, 3, 4, 5, 6, 7, 8]; + + // Test emit all + let emit = EmitTo::All; + let mut source = values.clone(); + let expected = values.clone(); + let actual = emit.take_needed(&mut source); + assert_eq!(actual, expected); + assert!(source.is_empty()); + + // Test emit first n + // n < source len + let emit = EmitTo::First(4); + let mut origin = values.clone(); + let expected = origin[0..4].to_vec(); + let rest_expected = origin[4..].to_vec(); + let actual = emit.take_needed(&mut origin); + assert_eq!(actual, expected); + assert_eq!(origin, rest_expected); + + // n > source len + let emit = EmitTo::First(9); + let mut origin = values.clone(); + let expected = values.clone(); + let actual = emit.take_needed(&mut origin); + assert_eq!(actual, expected); + assert!(origin.is_empty()); + } + + #[test] + fn test_take_need_from_blocks() { + let block1 = vec![1, 2, 3, 4]; + let block2 = vec![5, 6, 7, 8]; + + let mut values = VecBlocks::new(); + values.push_block(block1.clone()); + values.push_block(block2.clone()); + + // Test emit block + let emit = EmitTo::NextBlock(false); + let actual = + emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); + assert_eq!(actual, block1); + + let actual = + emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); + assert_eq!(actual, block2); + + let actual = + emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); + assert!(actual.is_empty()); + } } From 2b0796aee688c305b3e3f0448c2dd8d4dbd9cd17 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 23 Aug 2024 00:46:28 +0800 Subject: [PATCH 052/107] fix clippy. --- .../src/aggregate/groups_accumulator.rs | 8 ++++++-- .../src/aggregate/groups_accumulator/accumulate.rs | 6 +++--- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index cbb042e05804..fdd34d1b1faa 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -430,7 +430,7 @@ impl EmitToExt for EmitTo { match self { Self::All => { debug_assert!(matches!(mode, GroupStatesMode::Flat)); - blocks.pop_first_block().unwrap_or(Vec::new()) + blocks.pop_first_block().unwrap_or_default() } Self::First(n) => { debug_assert!(matches!(mode, GroupStatesMode::Flat)); @@ -446,7 +446,7 @@ impl EmitToExt for EmitTo { } Self::NextBlock(_) => { debug_assert!(matches!(mode, GroupStatesMode::Blocked(_))); - blocks.pop_first_block().unwrap_or(Vec::new()) + blocks.pop_first_block().unwrap_or_default() } } } @@ -656,6 +656,10 @@ impl VecBlocks { cur_len + prev_len } + + pub fn is_empty(&self) -> bool { + self.current.is_none() + } } fn get_filter_at_indices( 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 126b85d17458..4fb65dbc7b35 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -995,7 +995,7 @@ mod test { AccumulateTest::Original => { Fixture::accumulate_test( group_indices, - &values, + values, opt_filter, total_num_groups, NullState::new(), @@ -1004,7 +1004,7 @@ mod test { AccumulateTest::Flat => { Fixture::accumulate_test( group_indices, - &values, + values, opt_filter, total_num_groups, BlockedNullStateInFlatMode::new(), @@ -1013,7 +1013,7 @@ mod test { AccumulateTest::Blocked => { Fixture::accumulate_test( group_indices, - &values, + values, opt_filter, total_num_groups, BlockedNullStateInBlockedMode::new(), From 46b10b49b26d2a3727727c464312b4182be371dc Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 14:47:25 +0800 Subject: [PATCH 053/107] merge two modes to one. --- .../expr-common/src/groups_accumulator.rs | 14 + .../src/aggregate/groups_accumulator.rs | 110 +- .../groups_accumulator/accumulate.rs | 1500 ++++++++--------- .../aggregate/groups_accumulator/prim_op.rs | 86 +- datafusion/functions-aggregate/src/average.rs | 228 +-- datafusion/functions-aggregate/src/count.rs | 127 +- .../src/aggregates/group_values/mod.rs | 10 + .../src/aggregates/group_values/row.rs | 117 +- .../physical-plan/src/aggregates/row_hash.rs | 10 +- 9 files changed, 1043 insertions(+), 1159 deletions(-) diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index b52a4e76786d..f0463ae58112 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -164,6 +164,20 @@ pub trait GroupsAccumulator: Send { Ok(()) } + /// Switch the accumulator to flat or blocked mode. + /// You can see detail about the mode on [GroupStatesMode]. + /// + /// After switching mode, all data in previous mode 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(()) + } + /// Merges intermediate state (the output from [`Self::state`]) /// into this accumulator's current state. /// diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index fdd34d1b1faa..9560f7847c20 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -39,10 +39,8 @@ use datafusion_common::{ arrow_datafusion_err, utils::get_arrayref_at_indices, DataFusionError, Result, ScalarValue, }; +use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -use datafusion_expr_common::{ - accumulator::Accumulator, groups_accumulator::GroupStatesMode, -}; const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; @@ -394,11 +392,7 @@ pub trait EmitToExt { /// - For Emit::CurrentBlock, the first block will be taken and return. /// - For Emit::All and Emit::First, it will be only supported in `GroupStatesMode::Flat`, /// similar as `take_needed`. - fn take_needed_from_blocks( - &self, - blocks: &mut VecBlocks, - mode: GroupStatesMode, - ) -> Vec; + fn take_needed_from_blocks(&self, blocks: &mut VecBlocks) -> Vec; } impl EmitToExt for EmitTo { @@ -422,19 +416,14 @@ impl EmitToExt for EmitTo { } } - fn take_needed_from_blocks( - &self, - blocks: &mut VecBlocks, - mode: GroupStatesMode, - ) -> Vec { + fn take_needed_from_blocks(&self, blocks: &mut VecBlocks) -> Vec { match self { Self::All => { - debug_assert!(matches!(mode, GroupStatesMode::Flat)); + debug_assert!(blocks.num_blocks() == 1); blocks.pop_first_block().unwrap_or_default() } Self::First(n) => { - debug_assert!(matches!(mode, GroupStatesMode::Flat)); - + debug_assert!(blocks.num_blocks() == 1); let block = blocks.current_mut().unwrap(); let split_at = min(block.len(), *n); @@ -444,10 +433,7 @@ impl EmitToExt for EmitTo { std::mem::swap(block, &mut t); t } - Self::NextBlock(_) => { - debug_assert!(matches!(mode, GroupStatesMode::Blocked(_))); - blocks.pop_first_block().unwrap_or_default() - } + Self::NextBlock(_) => blocks.pop_first_block().unwrap_or_default(), } } } @@ -461,10 +447,19 @@ impl EmitToExt for EmitTo { pub struct BlockedGroupIndex { pub block_id: usize, pub block_offset: usize, + pub is_blocked: bool, } impl BlockedGroupIndex { - pub fn new(group_index: usize) -> Self { + pub fn new(group_index: usize, is_blocked: bool) -> Self { + if !is_blocked { + return Self { + block_id: 0, + block_offset: group_index, + is_blocked, + }; + } + let block_id = ((group_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; let block_offset = @@ -473,20 +468,28 @@ impl BlockedGroupIndex { Self { block_id, block_offset, + is_blocked, } } - pub fn new_from_parts(block_id: usize, block_offset: usize) -> Self { + pub fn new_from_parts( + block_id: usize, + block_offset: usize, + is_blocked: bool, + ) -> Self { Self { block_id, block_offset, + is_blocked, } } pub fn as_packed_index(&self) -> usize { - ((((self.block_id as u64) << 32) & BLOCKED_INDEX_HIGH_32_BITS_MASK) - | (self.block_offset as u64 & BLOCKED_INDEX_LOW_32_BITS_MASK)) - as usize + if self.is_blocked { + (((self.block_id as u64) << 32) | (self.block_offset as u64)) as usize + } else { + self.block_offset + } } } @@ -703,6 +706,25 @@ pub(crate) fn slice_and_maybe_filter( } } +pub fn ensure_enough_room_for_values( + values: &mut VecBlocks, + total_num_groups: usize, + block_size: Option, + default_value: T, +) { + match block_size { + Some(blk_size) => ensure_enough_room_for_blocked_values( + values, + total_num_groups, + blk_size, + default_value, + ), + None => { + ensure_enough_room_for_flat_values(values, total_num_groups, default_value) + } + } +} + /// Expend flat values to a big enough size for holding `total_num_groups` groups. /// /// For example, @@ -924,27 +946,27 @@ mod tests { assert!(origin.is_empty()); } - #[test] - fn test_take_need_from_blocks() { - let block1 = vec![1, 2, 3, 4]; - let block2 = vec![5, 6, 7, 8]; + // #[test] + // fn test_take_need_from_blocks() { + // let block1 = vec![1, 2, 3, 4]; + // let block2 = vec![5, 6, 7, 8]; - let mut values = VecBlocks::new(); - values.push_block(block1.clone()); - values.push_block(block2.clone()); + // let mut values = VecBlocks::new(); + // values.push_block(block1.clone()); + // values.push_block(block2.clone()); - // Test emit block - let emit = EmitTo::NextBlock(false); - let actual = - emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); - assert_eq!(actual, block1); + // // Test emit block + // let emit = EmitTo::NextBlock(false); + // let actual = + // emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); + // assert_eq!(actual, block1); - let actual = - emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); - assert_eq!(actual, block2); + // let actual = + // emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); + // assert_eq!(actual, block2); - let actual = - emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); - assert!(actual.is_empty()); - } + // let actual = + // emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); + // assert!(actual.is_empty()); + // } } 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 4fb65dbc7b35..3d82bc99f029 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 arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; -use datafusion_expr_common::groups_accumulator::{EmitTo, GroupStatesMode}; +use datafusion_expr_common::groups_accumulator::EmitTo; use crate::aggregate::groups_accumulator::{BlockedGroupIndex, Blocks}; /// Track the accumulator null state per row: if any values for that @@ -368,20 +368,20 @@ pub struct BlockedNullState { /// pass the filter yet for group `i` seen_values_blocks: Blocks, - mode: GroupStatesMode, + block_size: Option, } impl Default for BlockedNullState { fn default() -> Self { - Self::new(GroupStatesMode::Flat) + Self::new(None) } } impl BlockedNullState { - pub fn new(mode: GroupStatesMode) -> Self { + pub fn new(block_size: Option) -> Self { Self { seen_values_blocks: Blocks::new(), - mode, + block_size, } } @@ -395,7 +395,7 @@ impl BlockedNullState { } /// Similar as [NullState::accumulate] - pub fn accumulate_for_flat( + pub fn accumulate( &mut self, group_indices: &[usize], values: &PrimitiveArray, @@ -411,47 +411,17 @@ impl BlockedNullState { // ensure the seen_values is big enough (start everything at // "not seen" valid) - ensure_enough_room_for_flat_nulls( + ensure_enough_room_for_nulls( &mut self.seen_values_blocks, total_num_groups, + self.block_size, false, ); - let seen_values_blocks = &mut self.seen_values_blocks; - let block = seen_values_blocks.current_mut().unwrap(); - do_accumulate(group_indices, values, opt_filter, value_fn, |group_index| { - block.set_bit(group_index, true); - }); - } - - /// Similar as [NullState::accumulate] but designed for blocked version accumulator - pub fn accumulate_for_blocked( - &mut self, - group_indices: &[usize], - values: &PrimitiveArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - block_size: usize, - value_fn: F, - ) where - T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, - { - debug_assert!(total_num_groups > 0); - debug_assert_eq!(values.values().len(), group_indices.len()); - - // ensure the seen_values is big enough (start everything at - // "not seen" valid) - ensure_enough_room_for_blocked_nulls( - &mut self.seen_values_blocks, - total_num_groups, - block_size, - false, - ); + let is_blocked = self.block_size.is_some(); - let seen_values_blocks = &mut self.seen_values_blocks; do_accumulate(group_indices, values, opt_filter, value_fn, |group_index| { - let blocked_index = BlockedGroupIndex::new(group_index); + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); seen_values_blocks[blocked_index.block_id] .set_bit(blocked_index.block_offset, true); }); @@ -465,11 +435,11 @@ impl BlockedNullState { let nulls = match emit_to { EmitTo::All => { - debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); + debug_assert!(self.block_size.is_none()); self.seen_values_blocks.current_mut().unwrap().finish() } EmitTo::First(n) => { - debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); + debug_assert!(self.block_size.is_none()); // split off the first N values in seen_values // @@ -485,7 +455,7 @@ impl BlockedNullState { first_n_null } EmitTo::NextBlock(_) => { - debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); + debug_assert!(self.block_size.is_some()); let mut cur_blk = self.seen_values_blocks.pop_first_block().unwrap(); cur_blk.finish() } @@ -708,10 +678,32 @@ fn initialize_builder( builder } +fn ensure_enough_room_for_nulls( + builder_blocks: &mut Blocks, + total_num_groups: usize, + block_size: Option, + default_value: bool, +) { + match block_size { + Some(blk_size) => ensure_enough_room_for_blocked_nulls( + builder_blocks, + total_num_groups, + blk_size, + default_value, + ), + None => ensure_enough_room_for_flat_nulls( + builder_blocks, + total_num_groups, + default_value, + ), + } +} + /// Similar as the [initialize_builder] fn ensure_enough_room_for_flat_nulls( builder_blocks: &mut Blocks, total_num_groups: usize, + default_value: bool, ) { // It flat mode, we just a single builder, and grow it constantly. @@ -793,715 +785,715 @@ fn ensure_enough_room_for_blocked_nulls( #[cfg(test)] mod test { - use super::*; - - use arrow::array::UInt32Array; - use rand::{rngs::ThreadRng, Rng}; - use std::collections::HashSet; - - /// Null state's behaviors needed in accumulate test - trait TestNullState { - fn accumulate( - &mut self, - group_indices: &[usize], - values: &PrimitiveArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - accumulated_values: &mut Vec<(usize, T::Native)>, - ) where - T: ArrowPrimitiveType + Send; - - fn build_bool_buffer(&self) -> BooleanBuffer; - - fn build_null_buffer(&mut self) -> NullBuffer; - } - - /// The original `NullState` - impl TestNullState for NullState { - fn accumulate( - &mut self, - group_indices: &[usize], - values: &PrimitiveArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - accumulated_values: &mut Vec<(usize, T::Native)>, - ) where - T: ArrowPrimitiveType + Send, - { - self.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); - }, - ); - } - - fn build_bool_buffer(&self) -> BooleanBuffer { - self.seen_values.finish_cloned() - } - - fn build_null_buffer(&mut self) -> NullBuffer { - self.build(EmitTo::All) - } - } - - /// The new `BlockedNullState` in flat mode - struct BlockedNullStateInFlatMode(BlockedNullState); - - impl BlockedNullStateInFlatMode { - fn new() -> Self { - let null_state = BlockedNullState::new(GroupStatesMode::Flat); - - Self(null_state) - } - } - - impl TestNullState for BlockedNullStateInFlatMode { - fn accumulate( - &mut self, - group_indices: &[usize], - values: &PrimitiveArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - accumulated_values: &mut Vec<(usize, T::Native)>, - ) where - T: ArrowPrimitiveType + Send, - { - self.0.accumulate_for_flat( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_index, value)); - }, - ); - } - - fn build_bool_buffer(&self) -> BooleanBuffer { - self.0.seen_values_blocks.current().unwrap().finish_cloned() - } - - fn build_null_buffer(&mut self) -> NullBuffer { - self.0.build(EmitTo::All) - } - } - - /// The new `BlockedNullState` in blocked mode - struct BlockedNullStateInBlockedMode { - null_state: BlockedNullState, - block_size: usize, - } - - impl BlockedNullStateInBlockedMode { - fn new() -> Self { - let null_state = BlockedNullState::new(GroupStatesMode::Blocked(4)); - - Self { - null_state, - block_size: 4, - } - } - } - - impl TestNullState for BlockedNullStateInBlockedMode { - fn accumulate( - &mut self, - group_indices: &[usize], - values: &PrimitiveArray, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - accumulated_values: &mut Vec<(usize, T::Native)>, - ) where - T: ArrowPrimitiveType + Send, - { - // Convert group indices to blocked style - let blocked_indices = group_indices - .iter() - .map(|idx| { - let block_id = *idx / self.block_size; - let block_offset = *idx % self.block_size; - BlockedGroupIndex::new_from_parts(block_id, block_offset) - .as_packed_index() - }) - .collect::>(); - - self.null_state.accumulate_for_blocked( - &blocked_indices, - values, - opt_filter, - total_num_groups, - self.block_size, - |group_index, value| { - let blocked_index = BlockedGroupIndex::new(group_index); - let flat_index = blocked_index.block_id * self.block_size - + blocked_index.block_offset; - accumulated_values.push((flat_index, value)); - }, - ); - } - - fn build_bool_buffer(&self) -> BooleanBuffer { - let mut ret_builder = BooleanBufferBuilder::new(0); - for blk in self.null_state.seen_values_blocks.iter() { - let buf = blk.finish_cloned(); - for seen in buf.iter() { - ret_builder.append(seen); - } - } - ret_builder.finish() - } - - fn build_null_buffer(&mut self) -> NullBuffer { - let mut ret_builder = BooleanBufferBuilder::new(0); - loop { - let blk = self.null_state.build(EmitTo::NextBlock(false)); - if blk.is_empty() { - break; - } - - for seen in blk.iter() { - ret_builder.append(seen); - } - } - - NullBuffer::new(ret_builder.finish()) - } - } - - /// Accumulate test mode - /// - Original, test the original `NullState` - /// - Flat, test the `BlockedNullState` in flat mode - /// - Blocked, test the `BlockedNullState` in blocked mode - #[derive(Debug, Clone, Copy)] - enum AccumulateTest { - Original, - Flat, - Blocked, - } - - impl AccumulateTest { - fn run( - &self, - group_indices: &[usize], - values: &UInt32Array, - opt_filter: Option<&BooleanArray>, - total_num_groups: usize, - ) { - match self { - AccumulateTest::Original => { - Fixture::accumulate_test( - group_indices, - values, - opt_filter, - total_num_groups, - NullState::new(), - ); - } - AccumulateTest::Flat => { - Fixture::accumulate_test( - group_indices, - values, - opt_filter, - total_num_groups, - BlockedNullStateInFlatMode::new(), - ); - } - AccumulateTest::Blocked => { - Fixture::accumulate_test( - group_indices, - values, - opt_filter, - total_num_groups, - BlockedNullStateInBlockedMode::new(), - ); - } - } - } - } - - #[test] - fn accumulate_test_original() { - do_accumulate_test(AccumulateTest::Original); - } - - #[test] - fn accumulate_test_flat() { - do_accumulate_test(AccumulateTest::Flat); - } - - #[test] - fn accumulate_test_blocked() { - do_accumulate_test(AccumulateTest::Blocked); - } - - #[test] - fn accumulate_fuzz_test_original() { - do_accumulate_fuzz_test(AccumulateTest::Original); - } - - #[test] - fn accumulate_fuzz_test_flat() { - do_accumulate_fuzz_test(AccumulateTest::Flat); - } - - #[test] - fn accumulate_fuzz_test_blocked() { - do_accumulate_fuzz_test(AccumulateTest::Blocked); - } - - fn do_accumulate_test(accumulate_test: AccumulateTest) { - let group_indices = (0..100).collect(); - let values = (0..100).map(|i| (i + 1) * 10).collect(); - let values_with_nulls = (0..100) - .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) - .collect(); - - // default to every fifth value being false, every even - // being null - let filter: BooleanArray = (0..100) - .map(|i| { - let is_even = i % 2 == 0; - let is_fifth = i % 5 == 0; - if is_even { - None - } else if is_fifth { - Some(false) - } else { - Some(true) - } - }) - .collect(); - - Fixture { - group_indices, - values, - values_with_nulls, - filter, - accumulate_test, - } - .run() - } - - fn do_accumulate_fuzz_test(accumulate_test: AccumulateTest) { - let mut rng = rand::thread_rng(); - for _ in 0..100 { - Fixture::new_random(&mut rng, accumulate_test).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, - - /// tested null state for value test - accumulate_test: AccumulateTest, - } - - impl Fixture { - fn new_random(rng: &mut ThreadRng, accumulate_test: AccumulateTest) -> 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(); - - // 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, - accumulate_test, - } - } - - /// 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.run( - group_indices, - &values_array, - None, - total_num_groups, - ); - - // nulls, no filters - self.accumulate_test.run( - group_indices, - &values_with_nulls_array, - None, - total_num_groups, - ); - - // no nulls, filters - self.accumulate_test.run( - group_indices, - &values_array, - Some(filter), - total_num_groups, - ); - - // nulls, filters - self.accumulate_test.run( - group_indices, - &values_with_nulls_array, - Some(filter), - total_num_groups, - ); - } - - /// 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, - null_state_for_value_test: S, - ) { - Self::accumulate_values_test( - group_indices, - values, - opt_filter, - total_num_groups, - null_state_for_value_test, - ); - 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(); - - // TODO: test the `BlockedNullState` after supporting `accumulate_boolean` in it - Self::accumulate_boolean_test( - group_indices, - &boolean_values, - opt_filter, - total_num_groups, - ); - } - - /// 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, - mut null_state: S, - ) { - let mut accumulated_values = vec![]; - - null_state.accumulate( - group_indices, - values, - opt_filter, - total_num_groups, - &mut accumulated_values, - ); - - // 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_bool_buffer(); - 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_null_buffer(); - - 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, - ) { - let mut accumulated_values = vec![]; - let mut null_state = NullState::new(); - - null_state.accumulate_boolean( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, value| { - accumulated_values.push((group_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.seen_values.finish_cloned(); - 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); - - 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_ensure_room_for_blocked_nulls() { - let mut blocks: Blocks = Blocks::new(); - let block_size = 4; - - // 0 total_num_groups, should be no blocks - ensure_enough_room_for_blocked_nulls(&mut blocks, 0, block_size, false); - assert_eq!(blocks.num_blocks(), 0); - let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - assert_eq!(total_len, 0); - - // 0 -> 3 total_num_groups, blocks should look like: - // [d, d, d, empty] - ensure_enough_room_for_blocked_nulls(&mut blocks, 3, block_size, false); - assert_eq!(blocks.num_blocks(), 1); - let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - assert_eq!(total_len, 3); - - // 3 -> 8 total_num_groups, blocks should look like: - // [d, d, d, d], [d, d, d, d] - ensure_enough_room_for_blocked_nulls(&mut blocks, 8, block_size, false); - assert_eq!(blocks.num_blocks(), 2); - let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - assert_eq!(total_len, 8); - - // 8 -> 13 total_num_groups, blocks should look like: - // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] - ensure_enough_room_for_blocked_nulls(&mut blocks, 13, block_size, false); - assert_eq!(blocks.num_blocks(), 4); - let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - assert_eq!(total_len, 13); - } + // use super::*; + + // use arrow::array::UInt32Array; + // use rand::{rngs::ThreadRng, Rng}; + // use std::collections::HashSet; + + // /// Null state's behaviors needed in accumulate test + // trait TestNullState { + // fn accumulate( + // &mut self, + // group_indices: &[usize], + // values: &PrimitiveArray, + // opt_filter: Option<&BooleanArray>, + // total_num_groups: usize, + // accumulated_values: &mut Vec<(usize, T::Native)>, + // ) where + // T: ArrowPrimitiveType + Send; + + // fn build_bool_buffer(&self) -> BooleanBuffer; + + // fn build_null_buffer(&mut self) -> NullBuffer; + // } + + // /// The original `NullState` + // impl TestNullState for NullState { + // fn accumulate( + // &mut self, + // group_indices: &[usize], + // values: &PrimitiveArray, + // opt_filter: Option<&BooleanArray>, + // total_num_groups: usize, + // accumulated_values: &mut Vec<(usize, T::Native)>, + // ) where + // T: ArrowPrimitiveType + Send, + // { + // self.accumulate( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // |group_index, value| { + // accumulated_values.push((group_index, value)); + // }, + // ); + // } + + // fn build_bool_buffer(&self) -> BooleanBuffer { + // self.seen_values.finish_cloned() + // } + + // fn build_null_buffer(&mut self) -> NullBuffer { + // self.build(EmitTo::All) + // } + // } + + // /// The new `BlockedNullState` in flat mode + // struct BlockedNullStateInFlatMode(BlockedNullState); + + // impl BlockedNullStateInFlatMode { + // fn new() -> Self { + // let null_state = BlockedNullState::new(GroupStatesMode::Flat); + + // Self(null_state) + // } + // } + + // impl TestNullState for BlockedNullStateInFlatMode { + // fn accumulate( + // &mut self, + // group_indices: &[usize], + // values: &PrimitiveArray, + // opt_filter: Option<&BooleanArray>, + // total_num_groups: usize, + // accumulated_values: &mut Vec<(usize, T::Native)>, + // ) where + // T: ArrowPrimitiveType + Send, + // { + // self.0.accumulate_for_flat( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // |group_index, value| { + // accumulated_values.push((group_index, value)); + // }, + // ); + // } + + // fn build_bool_buffer(&self) -> BooleanBuffer { + // self.0.seen_values_blocks.current().unwrap().finish_cloned() + // } + + // fn build_null_buffer(&mut self) -> NullBuffer { + // self.0.build(EmitTo::All) + // } + // } + + // /// The new `BlockedNullState` in blocked mode + // struct BlockedNullStateInBlockedMode { + // null_state: BlockedNullState, + // block_size: usize, + // } + + // impl BlockedNullStateInBlockedMode { + // fn new() -> Self { + // let null_state = BlockedNullState::new(GroupStatesMode::Blocked(4)); + + // Self { + // null_state, + // block_size: 4, + // } + // } + // } + + // impl TestNullState for BlockedNullStateInBlockedMode { + // fn accumulate( + // &mut self, + // group_indices: &[usize], + // values: &PrimitiveArray, + // opt_filter: Option<&BooleanArray>, + // total_num_groups: usize, + // accumulated_values: &mut Vec<(usize, T::Native)>, + // ) where + // T: ArrowPrimitiveType + Send, + // { + // // Convert group indices to blocked style + // let blocked_indices = group_indices + // .iter() + // .map(|idx| { + // let block_id = *idx / self.block_size; + // let block_offset = *idx % self.block_size; + // BlockedGroupIndex::new_from_parts(block_id, block_offset) + // .as_packed_index() + // }) + // .collect::>(); + + // self.null_state.accumulate_for_blocked( + // &blocked_indices, + // values, + // opt_filter, + // total_num_groups, + // self.block_size, + // |group_index, value| { + // let blocked_index = BlockedGroupIndex::new(group_index); + // let flat_index = blocked_index.block_id * self.block_size + // + blocked_index.block_offset; + // accumulated_values.push((flat_index, value)); + // }, + // ); + // } + + // fn build_bool_buffer(&self) -> BooleanBuffer { + // let mut ret_builder = BooleanBufferBuilder::new(0); + // for blk in self.null_state.seen_values_blocks.iter() { + // let buf = blk.finish_cloned(); + // for seen in buf.iter() { + // ret_builder.append(seen); + // } + // } + // ret_builder.finish() + // } + + // fn build_null_buffer(&mut self) -> NullBuffer { + // let mut ret_builder = BooleanBufferBuilder::new(0); + // loop { + // let blk = self.null_state.build(EmitTo::NextBlock(false)); + // if blk.is_empty() { + // break; + // } + + // for seen in blk.iter() { + // ret_builder.append(seen); + // } + // } + + // NullBuffer::new(ret_builder.finish()) + // } + // } + + // /// Accumulate test mode + // /// - Original, test the original `NullState` + // /// - Flat, test the `BlockedNullState` in flat mode + // /// - Blocked, test the `BlockedNullState` in blocked mode + // #[derive(Debug, Clone, Copy)] + // enum AccumulateTest { + // Original, + // Flat, + // Blocked, + // } + + // impl AccumulateTest { + // fn run( + // &self, + // group_indices: &[usize], + // values: &UInt32Array, + // opt_filter: Option<&BooleanArray>, + // total_num_groups: usize, + // ) { + // match self { + // AccumulateTest::Original => { + // Fixture::accumulate_test( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // NullState::new(), + // ); + // } + // AccumulateTest::Flat => { + // Fixture::accumulate_test( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // BlockedNullStateInFlatMode::new(), + // ); + // } + // AccumulateTest::Blocked => { + // Fixture::accumulate_test( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // BlockedNullStateInBlockedMode::new(), + // ); + // } + // } + // } + // } + + // #[test] + // fn accumulate_test_original() { + // do_accumulate_test(AccumulateTest::Original); + // } + + // #[test] + // fn accumulate_test_flat() { + // do_accumulate_test(AccumulateTest::Flat); + // } + + // #[test] + // fn accumulate_test_blocked() { + // do_accumulate_test(AccumulateTest::Blocked); + // } + + // #[test] + // fn accumulate_fuzz_test_original() { + // do_accumulate_fuzz_test(AccumulateTest::Original); + // } + + // #[test] + // fn accumulate_fuzz_test_flat() { + // do_accumulate_fuzz_test(AccumulateTest::Flat); + // } + + // #[test] + // fn accumulate_fuzz_test_blocked() { + // do_accumulate_fuzz_test(AccumulateTest::Blocked); + // } + + // fn do_accumulate_test(accumulate_test: AccumulateTest) { + // let group_indices = (0..100).collect(); + // let values = (0..100).map(|i| (i + 1) * 10).collect(); + // let values_with_nulls = (0..100) + // .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) + // .collect(); + + // // default to every fifth value being false, every even + // // being null + // let filter: BooleanArray = (0..100) + // .map(|i| { + // let is_even = i % 2 == 0; + // let is_fifth = i % 5 == 0; + // if is_even { + // None + // } else if is_fifth { + // Some(false) + // } else { + // Some(true) + // } + // }) + // .collect(); + + // Fixture { + // group_indices, + // values, + // values_with_nulls, + // filter, + // accumulate_test, + // } + // .run() + // } + + // fn do_accumulate_fuzz_test(accumulate_test: AccumulateTest) { + // let mut rng = rand::thread_rng(); + // for _ in 0..100 { + // Fixture::new_random(&mut rng, accumulate_test).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, + + // /// tested null state for value test + // accumulate_test: AccumulateTest, + // } + + // impl Fixture { + // fn new_random(rng: &mut ThreadRng, accumulate_test: AccumulateTest) -> 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(); + + // // 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, + // accumulate_test, + // } + // } + + // /// 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.run( + // group_indices, + // &values_array, + // None, + // total_num_groups, + // ); + + // // nulls, no filters + // self.accumulate_test.run( + // group_indices, + // &values_with_nulls_array, + // None, + // total_num_groups, + // ); + + // // no nulls, filters + // self.accumulate_test.run( + // group_indices, + // &values_array, + // Some(filter), + // total_num_groups, + // ); + + // // nulls, filters + // self.accumulate_test.run( + // group_indices, + // &values_with_nulls_array, + // Some(filter), + // total_num_groups, + // ); + // } + + // /// 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, + // null_state_for_value_test: S, + // ) { + // Self::accumulate_values_test( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // null_state_for_value_test, + // ); + // 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(); + + // // TODO: test the `BlockedNullState` after supporting `accumulate_boolean` in it + // Self::accumulate_boolean_test( + // group_indices, + // &boolean_values, + // opt_filter, + // total_num_groups, + // ); + // } + + // /// 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, + // mut null_state: S, + // ) { + // let mut accumulated_values = vec![]; + + // null_state.accumulate( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // &mut accumulated_values, + // ); + + // // 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_bool_buffer(); + // 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_null_buffer(); + + // 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, + // ) { + // let mut accumulated_values = vec![]; + // let mut null_state = NullState::new(); + + // null_state.accumulate_boolean( + // group_indices, + // values, + // opt_filter, + // total_num_groups, + // |group_index, value| { + // accumulated_values.push((group_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.seen_values.finish_cloned(); + // 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); + + // 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_ensure_room_for_blocked_nulls() { + // let mut blocks: Blocks = Blocks::new(); + // let block_size = 4; + + // // 0 total_num_groups, should be no blocks + // ensure_enough_room_for_blocked_nulls(&mut blocks, 0, block_size, false); + // assert_eq!(blocks.num_blocks(), 0); + // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + // assert_eq!(total_len, 0); + + // // 0 -> 3 total_num_groups, blocks should look like: + // // [d, d, d, empty] + // ensure_enough_room_for_blocked_nulls(&mut blocks, 3, block_size, false); + // assert_eq!(blocks.num_blocks(), 1); + // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + // assert_eq!(total_len, 3); + + // // 3 -> 8 total_num_groups, blocks should look like: + // // [d, d, d, d], [d, d, d, d] + // ensure_enough_room_for_blocked_nulls(&mut blocks, 8, block_size, false); + // assert_eq!(blocks.num_blocks(), 2); + // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + // assert_eq!(total_len, 8); + + // // 8 -> 13 total_num_groups, blocks should look like: + // // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] + // ensure_enough_room_for_blocked_nulls(&mut blocks, 13, block_size, false); + // assert_eq!(blocks.num_blocks(), 4); + // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + // assert_eq!(total_len, 13); + // } } 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 8ef8a7267926..9186c82452a2 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 @@ -23,14 +23,11 @@ use arrow::compute; use arrow::datatypes::ArrowPrimitiveType; use arrow::datatypes::DataType; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; -use datafusion_expr_common::groups_accumulator::{ - EmitTo, GroupStatesMode, GroupsAccumulator, -}; +use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::{ - ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, - BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; /// An accumulator that implements a single operation over @@ -63,7 +60,7 @@ where /// Function that computes the primitive result prim_fn: F, - mode: GroupStatesMode, + block_size: Option, } impl PrimitiveGroupsAccumulator @@ -75,10 +72,10 @@ where Self { values_blocks: Blocks::new(), data_type: data_type.clone(), - null_state: BlockedNullState::new(GroupStatesMode::Flat), + null_state: BlockedNullState::new(None), starting_value: T::default_value(), prim_fn, - mode: GroupStatesMode::Flat, + block_size: None, } } @@ -109,57 +106,32 @@ where let values = values[0].as_primitive::(); // NullState dispatches / handles tracking nulls and groups that saw no values - match self.mode { - GroupStatesMode::Flat => { - // Ensure enough room in values - ensure_enough_room_for_flat_values( - &mut self.values_blocks, - total_num_groups, - self.starting_value, - ); - - let block = self.values_blocks.current_mut().unwrap(); - self.null_state.accumulate_for_flat( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, new_value| { - let value = &mut block[group_index]; - (self.prim_fn)(value, new_value); - }, - ); - } - GroupStatesMode::Blocked(blk_size) => { - // Ensure enough room in values - ensure_enough_room_for_blocked_values( - &mut self.values_blocks, - total_num_groups, - blk_size, - self.starting_value, - ); - - self.null_state.accumulate_for_blocked( - group_indices, - values, - opt_filter, - total_num_groups, - blk_size, - |group_index, new_value| { - let blocked_index = BlockedGroupIndex::new(group_index); - let value = &mut self.values_blocks[blocked_index.block_id] - [blocked_index.block_offset]; - (self.prim_fn)(value, new_value); - }, - ); - } - } + ensure_enough_room_for_values( + &mut self.values_blocks, + total_num_groups, + self.block_size, + self.starting_value, + ); + let is_blocked = self.block_size.is_some(); + + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let value = &mut self.values_blocks[blocked_index.block_id] + [blocked_index.block_offset]; + (self.prim_fn)(value, new_value); + }, + ); Ok(()) } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let values = emit_to.take_needed_from_blocks(&mut self.values_blocks, self.mode); + let values = emit_to.take_needed_from_blocks(&mut self.values_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()); @@ -249,10 +221,10 @@ where true } - fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.values_blocks.clear(); - self.null_state = BlockedNullState::new(mode); - self.mode = mode; + self.null_state = BlockedNullState::new(block_size); + self.block_size = block_size; Ok(()) } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index ab45eea7ca4e..70f05a6e8805 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -29,7 +29,6 @@ use arrow::datatypes::{ }; use datafusion_common::{exec_err, not_impl_err, Result, ScalarValue}; use datafusion_expr::function::{AccumulatorArgs, StateFieldsArgs}; -use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::type_coercion::aggregates::{avg_return_type, coerce_avg_type}; use datafusion_expr::utils::format_state_name; use datafusion_expr::Volatility::Immutable; @@ -42,8 +41,7 @@ use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls: filtered_null_mask, set_nulls, }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, - BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; use datafusion_functions_aggregate_common::utils::DecimalAverager; @@ -412,7 +410,7 @@ where /// Function that computes the final average (value / count) avg_fn: F, - mode: GroupStatesMode, + block_size: Option, } impl AvgGroupsAccumulator @@ -431,9 +429,9 @@ where sum_data_type: sum_data_type.clone(), counts: Blocks::new(), sums: Blocks::new(), - null_state: BlockedNullState::new(GroupStatesMode::Flat), + null_state: BlockedNullState::new(None), avg_fn, - mode: GroupStatesMode::Flat, + block_size: None, } } } @@ -458,72 +456,43 @@ where let values = values[0].as_primitive::(); // increment counts, update sums - match self.mode { - GroupStatesMode::Flat => { - ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); - ensure_enough_room_for_flat_values( - &mut self.sums, - total_num_groups, - T::default_value(), - ); - - let sum_block = self.sums.current_mut().unwrap(); - let count_block = self.counts.current_mut().unwrap(); - - self.null_state.accumulate_for_flat( - group_indices, - values, - opt_filter, - total_num_groups, - |group_index, new_value| { - let sum = &mut sum_block[group_index]; - let count = &mut count_block[group_index]; - - *sum = sum.add_wrapping(new_value); - *count += 1; - }, - ); - } - GroupStatesMode::Blocked(blk_size) => { - ensure_enough_room_for_blocked_values( - &mut self.counts, - total_num_groups, - blk_size, - 0, - ); - ensure_enough_room_for_blocked_values( - &mut self.sums, - total_num_groups, - blk_size, - T::default_value(), - ); - - self.null_state.accumulate_for_blocked( - group_indices, - values, - opt_filter, - total_num_groups, - blk_size, - |group_index, new_value| { - let blocked_index = BlockedGroupIndex::new(group_index); - let sum = &mut self.sums[blocked_index.block_id] - [blocked_index.block_offset]; - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; - - *sum = sum.add_wrapping(new_value); - *count += 1; - }, - ); - } - } + ensure_enough_room_for_values( + &mut self.counts, + total_num_groups, + self.block_size, + 0, + ); + ensure_enough_room_for_values( + &mut self.sums, + total_num_groups, + self.block_size, + T::default_value(), + ); + let is_blocked = self.block_size.is_some(); + + self.null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, new_value| { + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let sum = + &mut self.sums[blocked_index.block_id][blocked_index.block_offset]; + let count = + &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + + *sum = sum.add_wrapping(new_value); + *count += 1; + }, + ); Ok(()) } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); - let sums = emit_to.take_needed_from_blocks(&mut self.sums, self.mode); + let counts = emit_to.take_needed_from_blocks(&mut self.counts); + let sums = emit_to.take_needed_from_blocks(&mut self.sums); let nulls = self.null_state.build(emit_to); assert_eq!(nulls.len(), sums.len()); @@ -562,10 +531,10 @@ where let nulls = self.null_state.build(emit_to); let nulls = Some(nulls); - let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); + let counts = emit_to.take_needed_from_blocks(&mut self.counts); let counts = UInt64Array::new(counts.into(), nulls.clone()); // zero copy - let sums = emit_to.take_needed_from_blocks(&mut self.sums, self.mode); + let sums = emit_to.take_needed_from_blocks(&mut self.sums); let sums = PrimitiveArray::::new(sums.into(), nulls) // zero copy .with_data_type(self.sum_data_type.clone()); @@ -593,82 +562,45 @@ where let partial_sums = values[1].as_primitive::(); // update counts with partial counts + update sums - match self.mode { - GroupStatesMode::Flat => { - ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); - ensure_enough_room_for_flat_values( - &mut self.sums, - total_num_groups, - T::default_value(), - ); - - let count_block = self.counts.current_mut().unwrap(); - self.null_state.accumulate_for_flat( - group_indices, - partial_counts, - opt_filter, - total_num_groups, - |group_index, partial_count| { - let count = &mut count_block[group_index]; - *count += partial_count; - }, - ); - - let sum_block = self.sums.current_mut().unwrap(); - self.null_state.accumulate_for_flat( - group_indices, - partial_sums, - opt_filter, - total_num_groups, - |group_index, new_value: ::Native| { - let sum = &mut sum_block[group_index]; - *sum = sum.add_wrapping(new_value); - }, - ); - } - GroupStatesMode::Blocked(blk_size) => { - ensure_enough_room_for_blocked_values( - &mut self.counts, - total_num_groups, - blk_size, - 0, - ); - ensure_enough_room_for_blocked_values( - &mut self.sums, - total_num_groups, - blk_size, - T::default_value(), - ); - - self.null_state.accumulate_for_blocked( - group_indices, - partial_counts, - opt_filter, - total_num_groups, - blk_size, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; - *count += partial_count; - }, - ); - - self.null_state.accumulate_for_blocked( - group_indices, - partial_sums, - opt_filter, - total_num_groups, - blk_size, - |group_index, new_value: ::Native| { - let blocked_index = BlockedGroupIndex::new(group_index); - let sum = &mut self.sums[blocked_index.block_id] - [blocked_index.block_offset]; - *sum = sum.add_wrapping(new_value); - }, - ); - } - } + ensure_enough_room_for_values( + &mut self.counts, + total_num_groups, + self.block_size, + 0, + ); + ensure_enough_room_for_values( + &mut self.sums, + total_num_groups, + self.block_size, + T::default_value(), + ); + let is_blocked = self.block_size.is_some(); + + self.null_state.accumulate( + group_indices, + partial_counts, + opt_filter, + total_num_groups, + |group_index, partial_count| { + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let count = + &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + *count += partial_count; + }, + ); + + self.null_state.accumulate( + group_indices, + partial_sums, + opt_filter, + total_num_groups, + |group_index, new_value: ::Native| { + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let sum = + &mut self.sums[blocked_index.block_id][blocked_index.block_offset]; + *sum = sum.add_wrapping(new_value); + }, + ); Ok(()) } @@ -706,11 +638,11 @@ where true } - fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.counts.clear(); self.sums.clear(); - self.null_state = BlockedNullState::new(mode); - self.mode = mode; + self.null_state = BlockedNullState::new(block_size); + self.block_size = block_size; Ok(()) } diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index c607b442ccf3..8a7cddf57e2b 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -16,11 +16,9 @@ // under the License. use ahash::RandomState; -use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_blocked_values, ensure_enough_room_for_flat_values, - BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; use std::collections::HashSet; use std::ops::BitAnd; @@ -365,14 +363,14 @@ struct CountGroupsAccumulator { /// without copy. counts: VecBlocks, - mode: GroupStatesMode, + block_size: Option, } impl CountGroupsAccumulator { pub fn new() -> Self { Self { counts: Blocks::new(), - mode: GroupStatesMode::Flat, + block_size: None, } } } @@ -394,42 +392,25 @@ impl GroupsAccumulator for CountGroupsAccumulator { // Add one to each group's counter for each non null, non // filtered value - match self.mode { - GroupStatesMode::Flat => { - ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); - - let block = self.counts.current_mut().unwrap(); - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - let count = block.get_mut(group_index).unwrap(); - *count += 1; - }, - ); - } - GroupStatesMode::Blocked(blk_size) => { - ensure_enough_room_for_blocked_values( - &mut self.counts, - total_num_groups, - blk_size, - 0, - ); - - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - let blocked_index = BlockedGroupIndex::new(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; - *count += 1; - }, - ); - } - } + ensure_enough_room_for_values( + &mut self.counts, + total_num_groups, + self.block_size, + 0, + ); + let is_blocked = self.block_size.is_some(); + + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let count = + &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + *count += 1; + }, + ); Ok(()) } @@ -449,49 +430,31 @@ impl GroupsAccumulator for CountGroupsAccumulator { let values = &values[0]; // Adds the counts with the partial counts - match self.mode { - GroupStatesMode::Flat => { - ensure_enough_room_for_flat_values(&mut self.counts, total_num_groups, 0); - - let block = self.counts.current_mut().unwrap(); - do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let count = &mut block[group_index]; - *count += partial_count; - }, - ); - } - GroupStatesMode::Blocked(blk_size) => { - ensure_enough_room_for_blocked_values( - &mut self.counts, - total_num_groups, - blk_size, - 0, - ); - - do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; - *count += partial_count; - }, - ); - } - } + ensure_enough_room_for_values( + &mut self.counts, + total_num_groups, + self.block_size, + 0, + ); + let is_blocked = self.block_size.is_some(); + + do_count_merge_batch( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let count = + &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + *count += partial_count; + }, + ); Ok(()) } fn evaluate(&mut self, emit_to: EmitTo) -> Result { - let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); - + let counts = emit_to.take_needed_from_blocks(&mut self.counts); // Count is always non null (null inputs just don't contribute to the overall values) let nulls = None; let array = PrimitiveArray::::new(counts.into(), nulls); @@ -501,7 +464,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { // return arrays for counts fn state(&mut self, emit_to: EmitTo) -> Result> { - let counts = emit_to.take_needed_from_blocks(&mut self.counts, self.mode); + let counts = emit_to.take_needed_from_blocks(&mut self.counts); let counts: PrimitiveArray = Int64Array::from(counts); // zero copy, no nulls Ok(vec![Arc::new(counts) as ArrayRef]) @@ -580,9 +543,9 @@ impl GroupsAccumulator for CountGroupsAccumulator { true } - fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.counts.clear(); - self.mode = mode; + self.block_size = block_size; Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index 4e1b8c6ea0d0..f93959aa8ba6 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -71,6 +71,16 @@ pub trait GroupValues: Send { Ok(()) } + + 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(()) + } } pub fn new_group_values(schema: SchemaRef) -> Result> { diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 684e8412bfe8..54c36139148c 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -27,7 +27,6 @@ use arrow_schema::{DataType, SchemaRef}; use datafusion_common::hash_utils::create_hashes; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; -use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ BlockedGroupIndex, Blocks, @@ -75,7 +74,7 @@ pub struct GroupValuesRows { random_state: RandomState, /// Mode about current GroupValuesRows - mode: GroupStatesMode, + block_size: Option, } impl GroupValuesRows { @@ -103,7 +102,7 @@ impl GroupValuesRows { hashes_buffer: Default::default(), rows_buffer, random_state: Default::default(), - mode: GroupStatesMode::Flat, + block_size: None, }) } } @@ -118,11 +117,9 @@ impl GroupValues for GroupValuesRows { let mut group_values = mem::take(&mut self.group_values); if group_values.num_blocks() == 0 { - let block = match self.mode { - GroupStatesMode::Flat => self.row_converter.empty_rows(0, 0), - GroupStatesMode::Blocked(blk_size) => { - self.row_converter.empty_rows(blk_size, 0) - } + let block = match self.block_size { + Some(blk_size) => self.row_converter.empty_rows(blk_size, 0), + None => self.row_converter.empty_rows(0, 0), }; group_values.push_block(block); @@ -137,8 +134,8 @@ impl GroupValues for GroupValuesRows { batch_hashes.resize(n_rows, 0); create_hashes(cols, &self.random_state, batch_hashes)?; + let is_blocked = self.block_size.is_some(); for (row, &target_hash) in batch_hashes.iter().enumerate() { - let mode = self.mode; let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { // Somewhat surprisingly, this closure can be called even if the // hash doesn't match, so check the hash first with an integer @@ -151,18 +148,10 @@ impl GroupValues for GroupValuesRows { // verify that the group that we are inserting with hash is // actually the same key value as the group in // existing_idx (aka group_values @ row) - match mode { - GroupStatesMode::Flat => { - group_rows.row(row) - == group_values.current().unwrap().row(*group_idx) - } - GroupStatesMode::Blocked(_) => { - let blocked_index = BlockedGroupIndex::new(*group_idx); - group_rows.row(row) - == group_values[blocked_index.block_id] - .row(blocked_index.block_offset) - } - } + let blocked_index = BlockedGroupIndex::new(*group_idx, is_blocked); + group_rows.row(row) + == group_values[blocked_index.block_id] + .row(blocked_index.block_offset) }); let group_idx = match entry { @@ -171,34 +160,26 @@ impl GroupValues for GroupValuesRows { // 1.2 Need to create new entry for the group None => { // Add new entry to aggr_state and save newly created index - let group_idx = match mode { - GroupStatesMode::Flat => { - let blk = group_values.current_mut().unwrap(); - let group_idx = blk.num_rows(); - blk.push(group_rows.row(row)); - group_idx + if let Some(blk_size) = self.block_size { + if group_values.current().unwrap().num_rows() == blk_size { + // Use blk_size as offset cap, + // and old block's buffer size as buffer cap + let new_buf_cap = + rows_buffer_size(group_values.current().unwrap()); + let new_blk = + self.row_converter.empty_rows(blk_size, new_buf_cap); + group_values.push_block(new_blk); } - GroupStatesMode::Blocked(blk_size) => { - if group_values.current().unwrap().num_rows() == blk_size { - // Use blk_size as offset cap, - // and old block's buffer size as buffer cap - let new_buf_cap = - rows_buffer_size(group_values.current().unwrap()); - let new_blk = - self.row_converter.empty_rows(blk_size, new_buf_cap); - group_values.push_block(new_blk); - } + } - let blk_id = group_values.num_blocks() - 1; - let cur_blk = group_values.current_mut().unwrap(); - let blk_offset = cur_blk.num_rows(); - cur_blk.push(group_rows.row(row)); + let blk_id = group_values.num_blocks() - 1; + let cur_blk = group_values.current_mut().unwrap(); + let blk_offset = cur_blk.num_rows(); + cur_blk.push(group_rows.row(row)); - let blocked_index = - BlockedGroupIndex::new_from_parts(blk_id, blk_offset); - blocked_index.as_packed_index() - } - }; + let blocked_index = + BlockedGroupIndex::new_from_parts(blk_id, blk_offset, is_blocked); + let group_idx = blocked_index.as_packed_index(); // for hasher function, use precomputed hash value self.map.insert_accounted( @@ -206,6 +187,7 @@ impl GroupValues for GroupValuesRows { |(hash, _group_index)| *hash, &mut self.map_size, ); + group_idx } }; @@ -237,22 +219,20 @@ impl GroupValues for GroupValuesRows { } fn len(&self) -> usize { - match self.mode { - GroupStatesMode::Flat => self - .group_values - .current() - .map(|g| g.num_rows()) - .unwrap_or(0), - GroupStatesMode::Blocked(blk_size) => { - let num_blocks = self.group_values.num_blocks(); - if num_blocks == 0 { - return 0; - } - - (num_blocks - 1) * blk_size - + self.group_values.current().unwrap().num_rows() - } + let num_blocks = self.group_values.num_blocks(); + if num_blocks == 0 { + return 0; } + + let mut group_len = if let Some(blk_size) = self.block_size { + (num_blocks - 1) * blk_size + } else { + 0 + }; + + group_len = group_len + self.group_values.current().unwrap().num_rows(); + + group_len } fn emit(&mut self, emit_to: EmitTo) -> Result> { @@ -260,13 +240,13 @@ impl GroupValues for GroupValuesRows { let mut output = match emit_to { EmitTo::All => { - debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); + debug_assert!(self.block_size.is_none()); let blk = group_values.pop_first_block().unwrap(); self.row_converter.convert_rows(blk.into_iter())? } EmitTo::First(n) => { - debug_assert!(matches!(self.mode, GroupStatesMode::Flat)); + debug_assert!(self.block_size.is_none()); let blk = group_values.current_mut().unwrap(); let groups_rows = blk.iter().take(n); @@ -294,7 +274,7 @@ impl GroupValues for GroupValuesRows { output } EmitTo::NextBlock(true) => { - debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); + debug_assert!(self.block_size.is_some()); let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; @@ -302,13 +282,14 @@ impl GroupValues for GroupValuesRows { for bucket in self.map.iter() { // Decrement group index by n let group_idx = bucket.as_ref().1; - let old_blk_idx = BlockedGroupIndex::new(group_idx); + let old_blk_idx = BlockedGroupIndex::new(group_idx, true); match old_blk_idx.block_id.checked_sub(1) { // Group index was >= n, shift value down Some(new_blk_id) => { let new_group_idx = BlockedGroupIndex::new_from_parts( new_blk_id, old_blk_idx.block_offset, + true, ); bucket.as_mut().1 = new_group_idx.as_packed_index(); } @@ -320,8 +301,6 @@ impl GroupValues for GroupValuesRows { output } EmitTo::NextBlock(false) => { - debug_assert!(matches!(self.mode, GroupStatesMode::Blocked(_))); - let cur_blk = group_values.pop_first_block().unwrap(); self.row_converter.convert_rows(cur_blk.iter())? } @@ -360,10 +339,10 @@ impl GroupValues for GroupValuesRows { true } - fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { + fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.map.clear(); self.group_values.clear(); - self.mode = mode; + self.block_size = block_size; Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 1c63827f91a3..5390c8a9a0bb 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -625,10 +625,10 @@ fn maybe_enable_blocked_group_states( match (group_supports_blocked, accumulators_support_blocked) { (true, true) => { - group_values.switch_to_mode(GroupStatesMode::Blocked(block_size))?; - accumulators.iter_mut().try_for_each(|acc| { - acc.switch_to_mode(GroupStatesMode::Blocked(block_size)) - })?; + 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), @@ -1100,7 +1100,7 @@ impl GroupedHashAggregateStream { self.group_values.switch_to_mode(GroupStatesMode::Flat)?; self.accumulators .iter_mut() - .try_for_each(|acc| acc.switch_to_mode(GroupStatesMode::Flat))?; + .try_for_each(|acc| acc.alter_block_size(None))?; self.enable_blocked_group_states = false; } From aef6c490346a6a195f5a767427ca1d0a68a6811b Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 15:51:45 +0800 Subject: [PATCH 054/107] experiment. --- .../src/aggregate/groups_accumulator.rs | 66 +++++++++++-------- .../groups_accumulator/accumulate.rs | 63 +++++++++++------- .../aggregate/groups_accumulator/prim_op.rs | 12 ++-- datafusion/functions-aggregate/src/average.rs | 12 ++-- datafusion/functions-aggregate/src/count.rs | 14 ++-- .../src/aggregates/group_values/row.rs | 14 ++-- 6 files changed, 107 insertions(+), 74 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 9560f7847c20..21780aaeee64 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -26,8 +26,8 @@ pub mod prim_op; use std::{ cmp::min, collections::VecDeque, - fmt, iter, - ops::{Index, IndexMut}, + fmt::{self, Debug}, iter, + ops::{Index, IndexMut}, usize, }; use arrow::{ @@ -42,8 +42,8 @@ use datafusion_common::{ use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; -const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; +pub const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; +pub const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] /// @@ -451,28 +451,7 @@ pub struct BlockedGroupIndex { } impl BlockedGroupIndex { - pub fn new(group_index: usize, is_blocked: bool) -> Self { - if !is_blocked { - return Self { - block_id: 0, - block_offset: group_index, - is_blocked, - }; - } - - let block_id = - ((group_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - let block_offset = - ((group_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - - Self { - block_id, - block_offset, - is_blocked, - } - } - - pub fn new_from_parts( + pub fn new( block_id: usize, block_offset: usize, is_blocked: bool, @@ -484,6 +463,14 @@ impl BlockedGroupIndex { } } + pub fn parser(is_blocked: bool) -> Box { + if is_blocked { + Box::new(BlockedIndexParser) + } else { + Box::new(FlatIndexParser) + } + } + pub fn as_packed_index(&self) -> usize { if self.is_blocked { (((self.block_id as u64) << 32) | (self.block_offset as u64)) as usize @@ -493,6 +480,33 @@ impl BlockedGroupIndex { } } +pub trait GroupIndexParser: Send + Debug { + fn parse(&self, raw_index: usize) -> BlockedGroupIndex; +} + +#[derive(Debug)] +struct FlatIndexParser; + +impl GroupIndexParser for FlatIndexParser { + fn parse(&self, raw_index: usize) -> BlockedGroupIndex { + BlockedGroupIndex::new(0, raw_index, false) + } +} + +#[derive(Debug)] +struct BlockedIndexParser; + +impl GroupIndexParser for BlockedIndexParser { + fn parse(&self, raw_index: usize) -> BlockedGroupIndex { + let block_id = + ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + let block_offset = + ((raw_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + + BlockedGroupIndex::new(block_id, block_offset, true) + } +} + /// The basic data structure for blocked aggregation intermediate results /// /// The reason why not use `VecDeque` directly: 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 3d82bc99f029..b59b1053c8c4 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -25,7 +25,9 @@ use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; -use crate::aggregate::groups_accumulator::{BlockedGroupIndex, Blocks}; +use crate::aggregate::groups_accumulator::{ + BlockedGroupIndex, Blocks, GroupIndexParser, +}; /// 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. /// @@ -369,6 +371,8 @@ pub struct BlockedNullState { seen_values_blocks: Blocks, block_size: Option, + + group_index_parser: Box, } impl Default for BlockedNullState { @@ -379,9 +383,12 @@ impl Default for BlockedNullState { impl BlockedNullState { pub fn new(block_size: Option) -> Self { + let group_index_parser = BlockedGroupIndex::parser(block_size.is_some()); + Self { seen_values_blocks: Blocks::new(), block_size, + group_index_parser, } } @@ -404,7 +411,7 @@ impl BlockedNullState { value_fn: F, ) where T: ArrowPrimitiveType + Send, - F: FnMut(usize, T::Native) + Send, + F: FnMut(&BlockedGroupIndex, T::Native) + Send, { debug_assert!(total_num_groups > 0); debug_assert_eq!(values.values().len(), group_indices.len()); @@ -418,13 +425,18 @@ impl BlockedNullState { false, ); let seen_values_blocks = &mut self.seen_values_blocks; - let is_blocked = self.block_size.is_some(); - - do_accumulate(group_indices, values, opt_filter, value_fn, |group_index| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); - seen_values_blocks[blocked_index.block_id] - .set_bit(blocked_index.block_offset, true); - }); + let group_index_parser = self.group_index_parser.as_ref(); + + do_accumulate( + group_indices, + values, + opt_filter, + group_index_parser, + value_fn, + |index: &BlockedGroupIndex| { + seen_values_blocks[index.block_id].set_bit(index.block_offset, true); + }, + ); } /// Similar as [NullState::build] but support the blocked version accumulator @@ -480,7 +492,7 @@ pub fn accumulate_indices( opt_filter: Option<&BooleanArray>, mut index_fn: F, ) where - F: FnMut(usize) + Send, + F: FnMut(usize), { match (nulls, opt_filter) { (None, None) => { @@ -560,22 +572,23 @@ fn do_accumulate( group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, + group_index_parser: &dyn GroupIndexParser, mut value_fn: F1, mut set_valid_fn: F2, ) where T: ArrowPrimitiveType + Send, - F1: FnMut(usize, T::Native) + Send, - F2: FnMut(usize) + Send, + F1: FnMut(&BlockedGroupIndex, T::Native) + Send, + F2: FnMut(&BlockedGroupIndex) + Send, { let data: &[T::Native] = values.values(); - match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { let iter = group_indices.iter().zip(data.iter()); for (&group_index, &new_value) in iter { - set_valid_fn(group_index); - value_fn(group_index, new_value); + let blocked_index = group_index_parser.parse(group_index); + set_valid_fn(&blocked_index); + value_fn(&blocked_index, new_value); } } // nulls, no filter @@ -601,8 +614,9 @@ fn do_accumulate( // valid bit was set, real value let is_valid = (mask & index_mask) != 0; if is_valid { - set_valid_fn(group_index); - value_fn(group_index, new_value); + let blocked_index = group_index_parser.parse(group_index); + set_valid_fn(&blocked_index); + value_fn(&blocked_index, new_value); } index_mask <<= 1; }, @@ -618,8 +632,9 @@ fn do_accumulate( .for_each(|(i, (&group_index, &new_value))| { let is_valid = remainder_bits & (1 << i) != 0; if is_valid { - set_valid_fn(group_index); - value_fn(group_index, new_value); + let blocked_index = group_index_parser.parse(group_index); + set_valid_fn(&blocked_index); + value_fn(&blocked_index, new_value); } }); } @@ -635,8 +650,9 @@ fn do_accumulate( .zip(filter.iter()) .for_each(|((&group_index, &new_value), filter_value)| { if let Some(true) = filter_value { - set_valid_fn(group_index); - value_fn(group_index, new_value); + let blocked_index = group_index_parser.parse(group_index); + set_valid_fn(&blocked_index); + value_fn(&blocked_index, new_value); } }) } @@ -653,8 +669,9 @@ fn do_accumulate( .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - set_valid_fn(group_index); - value_fn(group_index, new_value) + let blocked_index = group_index_parser.parse(group_index); + set_valid_fn(&blocked_index); + value_fn(&blocked_index, new_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 9186c82452a2..c0216ea1935b 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 @@ -27,7 +27,7 @@ use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, Blocks, EmitToExt, VecBlocks, }; /// An accumulator that implements a single operation over @@ -112,17 +112,15 @@ where self.block_size, self.starting_value, ); - let is_blocked = self.block_size.is_some(); - + self.null_state.accumulate( group_indices, values, opt_filter, total_num_groups, - |group_index, new_value| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); - let value = &mut self.values_blocks[blocked_index.block_id] - [blocked_index.block_offset]; + |index, new_value| { + let value = &mut self.values_blocks[index.block_id] + [index.block_offset]; (self.prim_fn)(value, new_value); }, ); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 70f05a6e8805..f8b7bd8bed00 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -41,7 +41,7 @@ use datafusion_functions_aggregate_common::aggregate::groups_accumulator::nulls: filtered_null_mask, set_nulls, }; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, Blocks, EmitToExt, VecBlocks, }; use datafusion_functions_aggregate_common::utils::DecimalAverager; @@ -468,15 +468,13 @@ where self.block_size, T::default_value(), ); - let is_blocked = self.block_size.is_some(); self.null_state.accumulate( group_indices, values, opt_filter, total_num_groups, - |group_index, new_value| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + |blocked_index, new_value| { let sum = &mut self.sums[blocked_index.block_id][blocked_index.block_offset]; let count = @@ -581,8 +579,7 @@ where partial_counts, opt_filter, total_num_groups, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + |blocked_index, partial_count| { let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += partial_count; @@ -594,8 +591,7 @@ where partial_sums, opt_filter, total_num_groups, - |group_index, new_value: ::Native| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + |blocked_index, new_value: ::Native| { let sum = &mut self.sums[blocked_index.block_id][blocked_index.block_offset]; *sum = sum.add_wrapping(new_value); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 8a7cddf57e2b..2c366cc1bd8c 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -18,7 +18,7 @@ use ahash::RandomState; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, GroupIndexParser, VecBlocks }; use std::collections::HashSet; use std::ops::BitAnd; @@ -364,6 +364,8 @@ struct CountGroupsAccumulator { counts: VecBlocks, block_size: Option, + + group_index_parser: Box, } impl CountGroupsAccumulator { @@ -371,6 +373,7 @@ impl CountGroupsAccumulator { Self { counts: Blocks::new(), block_size: None, + group_index_parser: BlockedGroupIndex::parser(false), } } } @@ -398,14 +401,14 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.block_size, 0, ); - let is_blocked = self.block_size.is_some(); + let group_index_parser = self.group_index_parser.as_ref(); accumulate_indices( group_indices, values.logical_nulls().as_ref(), opt_filter, |group_index| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let blocked_index = group_index_parser.parse(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += 1; @@ -436,14 +439,14 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.block_size, 0, ); - let is_blocked = self.block_size.is_some(); + let group_index_parser = self.group_index_parser.as_ref(); do_count_merge_batch( values, group_indices, opt_filter, |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new(group_index, is_blocked); + let blocked_index = group_index_parser.parse(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += partial_count; @@ -546,6 +549,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.counts.clear(); self.block_size = block_size; + self.group_index_parser = BlockedGroupIndex::parser(block_size.is_some()); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 54c36139148c..974f462913f9 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -29,7 +29,7 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - BlockedGroupIndex, Blocks, + BlockedGroupIndex, Blocks, GroupIndexParser, }; use hashbrown::raw::RawTable; @@ -75,6 +75,8 @@ pub struct GroupValuesRows { /// Mode about current GroupValuesRows block_size: Option, + + group_index_parser: Box, } impl GroupValuesRows { @@ -103,6 +105,7 @@ impl GroupValuesRows { rows_buffer, random_state: Default::default(), block_size: None, + group_index_parser: BlockedGroupIndex::parser(false), }) } } @@ -148,7 +151,7 @@ impl GroupValues for GroupValuesRows { // verify that the group that we are inserting with hash is // actually the same key value as the group in // existing_idx (aka group_values @ row) - let blocked_index = BlockedGroupIndex::new(*group_idx, is_blocked); + let blocked_index = self.group_index_parser.parse(*group_idx); group_rows.row(row) == group_values[blocked_index.block_id] .row(blocked_index.block_offset) @@ -178,7 +181,7 @@ impl GroupValues for GroupValuesRows { cur_blk.push(group_rows.row(row)); let blocked_index = - BlockedGroupIndex::new_from_parts(blk_id, blk_offset, is_blocked); + BlockedGroupIndex::new(blk_id, blk_offset, is_blocked); let group_idx = blocked_index.as_packed_index(); // for hasher function, use precomputed hash value @@ -282,11 +285,11 @@ impl GroupValues for GroupValuesRows { for bucket in self.map.iter() { // Decrement group index by n let group_idx = bucket.as_ref().1; - let old_blk_idx = BlockedGroupIndex::new(group_idx, true); + let old_blk_idx = self.group_index_parser.parse(group_idx); match old_blk_idx.block_id.checked_sub(1) { // Group index was >= n, shift value down Some(new_blk_id) => { - let new_group_idx = BlockedGroupIndex::new_from_parts( + let new_group_idx = BlockedGroupIndex::new( new_blk_id, old_blk_idx.block_offset, true, @@ -343,6 +346,7 @@ impl GroupValues for GroupValuesRows { self.map.clear(); self.group_values.clear(); self.block_size = block_size; + self.group_index_parser = BlockedGroupIndex::parser(block_size.is_some()); Ok(()) } From 127a6e78d6aadde3536c5d8943d4c499a285568e Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 19:03:58 +0800 Subject: [PATCH 055/107] use function point to replace trait. --- .../src/aggregate/groups_accumulator.rs | 66 ++++++++----------- .../groups_accumulator/accumulate.rs | 32 +++++---- datafusion/functions-aggregate/src/average.rs | 1 - datafusion/functions-aggregate/src/count.rs | 32 ++++++--- .../src/aggregates/group_values/row.rs | 33 +++++++--- 5 files changed, 91 insertions(+), 73 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 21780aaeee64..ca7b778bed90 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -26,8 +26,10 @@ pub mod prim_op; use std::{ cmp::min, collections::VecDeque, - fmt::{self, Debug}, iter, - ops::{Index, IndexMut}, usize, + fmt::{self, Debug}, + iter, + ops::{Index, IndexMut}, + usize, }; use arrow::{ @@ -451,11 +453,8 @@ pub struct BlockedGroupIndex { } impl BlockedGroupIndex { - pub fn new( - block_id: usize, - block_offset: usize, - is_blocked: bool, - ) -> Self { + #[inline] + pub fn new(block_id: usize, block_offset: usize, is_blocked: bool) -> Self { Self { block_id, block_offset, @@ -463,13 +462,27 @@ impl BlockedGroupIndex { } } - pub fn parser(is_blocked: bool) -> Box { - if is_blocked { - Box::new(BlockedIndexParser) - } else { - Box::new(FlatIndexParser) + #[inline] + pub fn new_flat(raw_index: usize) -> Self { + Self { + block_id: 0, + block_offset: raw_index, + is_blocked: false, } - } + } + + #[inline] + pub fn new_blocked(raw_index: usize) -> Self { + let block_id = + ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + let block_offset = ((raw_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + + Self { + block_id, + block_offset, + is_blocked: true, + } + } pub fn as_packed_index(&self) -> usize { if self.is_blocked { @@ -480,33 +493,6 @@ impl BlockedGroupIndex { } } -pub trait GroupIndexParser: Send + Debug { - fn parse(&self, raw_index: usize) -> BlockedGroupIndex; -} - -#[derive(Debug)] -struct FlatIndexParser; - -impl GroupIndexParser for FlatIndexParser { - fn parse(&self, raw_index: usize) -> BlockedGroupIndex { - BlockedGroupIndex::new(0, raw_index, false) - } -} - -#[derive(Debug)] -struct BlockedIndexParser; - -impl GroupIndexParser for BlockedIndexParser { - fn parse(&self, raw_index: usize) -> BlockedGroupIndex { - let block_id = - ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - let block_offset = - ((raw_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - - BlockedGroupIndex::new(block_id, block_offset, true) - } -} - /// The basic data structure for blocked aggregation intermediate results /// /// The reason why not use `VecDeque` directly: 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 b59b1053c8c4..a3ca45233d90 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -26,7 +26,7 @@ use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; use crate::aggregate::groups_accumulator::{ - BlockedGroupIndex, Blocks, GroupIndexParser, + BlockedGroupIndex, 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. @@ -371,8 +371,6 @@ pub struct BlockedNullState { seen_values_blocks: Blocks, block_size: Option, - - group_index_parser: Box, } impl Default for BlockedNullState { @@ -383,12 +381,9 @@ impl Default for BlockedNullState { impl BlockedNullState { pub fn new(block_size: Option) -> Self { - let group_index_parser = BlockedGroupIndex::parser(block_size.is_some()); - Self { seen_values_blocks: Blocks::new(), block_size, - group_index_parser, } } @@ -425,13 +420,22 @@ impl BlockedNullState { false, ); let seen_values_blocks = &mut self.seen_values_blocks; - let group_index_parser = self.group_index_parser.as_ref(); + + let group_index_parse_fn = if self.block_size.is_some() { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_blocked(raw_index) + } + } else { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_flat(raw_index) + } + }; do_accumulate( group_indices, values, opt_filter, - group_index_parser, + group_index_parse_fn, value_fn, |index: &BlockedGroupIndex| { seen_values_blocks[index.block_id].set_bit(index.block_offset, true); @@ -572,7 +576,7 @@ fn do_accumulate( group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, - group_index_parser: &dyn GroupIndexParser, + group_index_parse_fn: fn(usize) -> BlockedGroupIndex, mut value_fn: F1, mut set_valid_fn: F2, ) where @@ -586,7 +590,7 @@ fn do_accumulate( (false, None) => { let iter = group_indices.iter().zip(data.iter()); for (&group_index, &new_value) in iter { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -614,7 +618,7 @@ fn do_accumulate( // valid bit was set, real value let is_valid = (mask & index_mask) != 0; if is_valid { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -632,7 +636,7 @@ fn do_accumulate( .for_each(|(i, (&group_index, &new_value))| { let is_valid = remainder_bits & (1 << i) != 0; if is_valid { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -650,7 +654,7 @@ fn do_accumulate( .zip(filter.iter()) .for_each(|((&group_index, &new_value), filter_value)| { if let Some(true) = filter_value { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -669,7 +673,7 @@ fn do_accumulate( .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value) } diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index f8b7bd8bed00..5c6d447e4b15 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -572,7 +572,6 @@ where self.block_size, T::default_value(), ); - let is_blocked = self.block_size.is_some(); self.null_state.accumulate( group_indices, diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 2c366cc1bd8c..66aefc075cc3 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -18,7 +18,7 @@ use ahash::RandomState; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, GroupIndexParser, VecBlocks + ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks }; use std::collections::HashSet; use std::ops::BitAnd; @@ -364,8 +364,6 @@ struct CountGroupsAccumulator { counts: VecBlocks, block_size: Option, - - group_index_parser: Box, } impl CountGroupsAccumulator { @@ -373,7 +371,6 @@ impl CountGroupsAccumulator { Self { counts: Blocks::new(), block_size: None, - group_index_parser: BlockedGroupIndex::parser(false), } } } @@ -402,13 +399,22 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - let group_index_parser = self.group_index_parser.as_ref(); + let group_index_parse_fn = if self.block_size.is_some() { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_blocked(raw_index) + } + } else { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_flat(raw_index) + } + }; + accumulate_indices( group_indices, values.logical_nulls().as_ref(), opt_filter, |group_index| { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += 1; @@ -440,13 +446,22 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - let group_index_parser = self.group_index_parser.as_ref(); + let group_index_parse_fn = if self.block_size.is_some() { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_blocked(raw_index) + } + } else { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_flat(raw_index) + } + }; + do_count_merge_batch( values, group_indices, opt_filter, |group_index, partial_count| { - let blocked_index = group_index_parser.parse(group_index); + let blocked_index = group_index_parse_fn(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += partial_count; @@ -549,7 +564,6 @@ impl GroupsAccumulator for CountGroupsAccumulator { fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.counts.clear(); self.block_size = block_size; - self.group_index_parser = BlockedGroupIndex::parser(block_size.is_some()); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 974f462913f9..00373cfedf84 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -29,7 +29,7 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - BlockedGroupIndex, Blocks, GroupIndexParser, + BlockedGroupIndex, Blocks, }; use hashbrown::raw::RawTable; @@ -75,8 +75,6 @@ pub struct GroupValuesRows { /// Mode about current GroupValuesRows block_size: Option, - - group_index_parser: Box, } impl GroupValuesRows { @@ -105,7 +103,6 @@ impl GroupValuesRows { rows_buffer, random_state: Default::default(), block_size: None, - group_index_parser: BlockedGroupIndex::parser(false), }) } } @@ -137,7 +134,16 @@ impl GroupValues for GroupValuesRows { batch_hashes.resize(n_rows, 0); create_hashes(cols, &self.random_state, batch_hashes)?; - let is_blocked = self.block_size.is_some(); + let group_index_parse_fn = if self.block_size.is_some() { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_blocked(raw_index) + } + } else { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_flat(raw_index) + } + }; + for (row, &target_hash) in batch_hashes.iter().enumerate() { let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { // Somewhat surprisingly, this closure can be called even if the @@ -151,7 +157,7 @@ impl GroupValues for GroupValuesRows { // verify that the group that we are inserting with hash is // actually the same key value as the group in // existing_idx (aka group_values @ row) - let blocked_index = self.group_index_parser.parse(*group_idx); + let blocked_index = group_index_parse_fn(*group_idx); group_rows.row(row) == group_values[blocked_index.block_id] .row(blocked_index.block_offset) @@ -181,7 +187,7 @@ impl GroupValues for GroupValuesRows { cur_blk.push(group_rows.row(row)); let blocked_index = - BlockedGroupIndex::new(blk_id, blk_offset, is_blocked); + BlockedGroupIndex::new(blk_id, blk_offset, self.block_size.is_some()); let group_idx = blocked_index.as_packed_index(); // for hasher function, use precomputed hash value @@ -281,11 +287,21 @@ impl GroupValues for GroupValuesRows { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; + let group_index_parse_fn = if self.block_size.is_some() { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_blocked(raw_index) + } + } else { + |raw_index: usize| -> BlockedGroupIndex { + BlockedGroupIndex::new_flat(raw_index) + } + }; + unsafe { for bucket in self.map.iter() { // Decrement group index by n let group_idx = bucket.as_ref().1; - let old_blk_idx = self.group_index_parser.parse(group_idx); + let old_blk_idx = group_index_parse_fn(group_idx); match old_blk_idx.block_id.checked_sub(1) { // Group index was >= n, shift value down Some(new_blk_id) => { @@ -346,7 +362,6 @@ impl GroupValues for GroupValuesRows { self.map.clear(); self.group_values.clear(); self.block_size = block_size; - self.group_index_parser = BlockedGroupIndex::parser(block_size.is_some()); Ok(()) } From 9bffb4a0c2047b44f92d39d75757bc24cb3db1d3 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 19:23:31 +0800 Subject: [PATCH 056/107] simplify function pointer. --- .../groups_accumulator/accumulate.rs | 8 ++----- datafusion/functions-aggregate/src/count.rs | 16 ++++--------- .../src/aggregates/group_values/row.rs | 23 ++++++++----------- 3 files changed, 15 insertions(+), 32 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 a3ca45233d90..462ff9230a6c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -422,13 +422,9 @@ impl BlockedNullState { let seen_values_blocks = &mut self.seen_values_blocks; let group_index_parse_fn = if self.block_size.is_some() { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_blocked(raw_index) - } + BlockedGroupIndex::new_blocked } else { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_flat(raw_index) - } + BlockedGroupIndex::new_flat }; do_accumulate( diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 66aefc075cc3..cb23979d1277 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -400,13 +400,9 @@ impl GroupsAccumulator for CountGroupsAccumulator { ); let group_index_parse_fn = if self.block_size.is_some() { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_blocked(raw_index) - } + BlockedGroupIndex::new_blocked } else { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_flat(raw_index) - } + BlockedGroupIndex::new_flat }; accumulate_indices( @@ -447,13 +443,9 @@ impl GroupsAccumulator for CountGroupsAccumulator { ); let group_index_parse_fn = if self.block_size.is_some() { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_blocked(raw_index) - } + BlockedGroupIndex::new_blocked } else { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_flat(raw_index) - } + BlockedGroupIndex::new_flat }; do_count_merge_batch( diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 00373cfedf84..864fc3481525 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -135,13 +135,9 @@ impl GroupValues for GroupValuesRows { create_hashes(cols, &self.random_state, batch_hashes)?; let group_index_parse_fn = if self.block_size.is_some() { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_blocked(raw_index) - } + BlockedGroupIndex::new_blocked } else { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_flat(raw_index) - } + BlockedGroupIndex::new_flat }; for (row, &target_hash) in batch_hashes.iter().enumerate() { @@ -186,8 +182,11 @@ impl GroupValues for GroupValuesRows { let blk_offset = cur_blk.num_rows(); cur_blk.push(group_rows.row(row)); - let blocked_index = - BlockedGroupIndex::new(blk_id, blk_offset, self.block_size.is_some()); + let blocked_index = BlockedGroupIndex::new( + blk_id, + blk_offset, + self.block_size.is_some(), + ); let group_idx = blocked_index.as_packed_index(); // for hasher function, use precomputed hash value @@ -288,13 +287,9 @@ impl GroupValues for GroupValuesRows { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; let group_index_parse_fn = if self.block_size.is_some() { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_blocked(raw_index) - } + BlockedGroupIndex::new_blocked } else { - |raw_index: usize| -> BlockedGroupIndex { - BlockedGroupIndex::new_flat(raw_index) - } + BlockedGroupIndex::new_flat }; unsafe { From c8c0fee9978b5018a3855854bccc8960f520c44d Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 19:36:52 +0800 Subject: [PATCH 057/107] init the function point during new. --- .../groups_accumulator/accumulate.rs | 17 ++++++++------ .../aggregate/groups_accumulator/prim_op.rs | 2 +- datafusion/functions-aggregate/src/count.rs | 22 +++++++++---------- .../src/aggregates/group_values/row.rs | 21 +++++++++--------- 4 files changed, 32 insertions(+), 30 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 462ff9230a6c..6cc72bd00877 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -371,6 +371,8 @@ pub struct BlockedNullState { seen_values_blocks: Blocks, block_size: Option, + + group_index_parse_fn: fn(usize) -> BlockedGroupIndex, } impl Default for BlockedNullState { @@ -381,9 +383,16 @@ impl Default for BlockedNullState { impl BlockedNullState { pub fn new(block_size: Option) -> Self { + let group_index_parse_fn = if block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; + Self { seen_values_blocks: Blocks::new(), block_size, + group_index_parse_fn, } } @@ -421,17 +430,11 @@ impl BlockedNullState { ); let seen_values_blocks = &mut self.seen_values_blocks; - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - do_accumulate( group_indices, values, opt_filter, - group_index_parse_fn, + self.group_index_parse_fn, value_fn, |index: &BlockedGroupIndex| { seen_values_blocks[index.block_id].set_bit(index.block_offset, true); 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 c0216ea1935b..370e4df6eb80 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 @@ -27,7 +27,7 @@ use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, Blocks, EmitToExt, VecBlocks }; /// An accumulator that implements a single operation over diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index cb23979d1277..a87bbe27396e 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -364,6 +364,8 @@ struct CountGroupsAccumulator { counts: VecBlocks, block_size: Option, + + group_index_parse_fn: fn(usize) -> BlockedGroupIndex, } impl CountGroupsAccumulator { @@ -371,6 +373,7 @@ impl CountGroupsAccumulator { Self { counts: Blocks::new(), block_size: None, + group_index_parse_fn: BlockedGroupIndex::new_flat, } } } @@ -399,12 +402,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - + let group_index_parse_fn = self.group_index_parse_fn; accumulate_indices( group_indices, values.logical_nulls().as_ref(), @@ -442,12 +440,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - + let group_index_parse_fn = self.group_index_parse_fn; do_count_merge_batch( values, group_indices, @@ -556,6 +549,11 @@ impl GroupsAccumulator for CountGroupsAccumulator { fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.counts.clear(); self.block_size = block_size; + self.group_index_parse_fn = if block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 864fc3481525..f710af3214f0 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -75,6 +75,8 @@ pub struct GroupValuesRows { /// Mode about current GroupValuesRows block_size: Option, + + group_index_parse_fn: fn(usize) -> BlockedGroupIndex, } impl GroupValuesRows { @@ -93,6 +95,7 @@ impl GroupValuesRows { let starting_data_capacity = 64 * starting_rows_capacity; let rows_buffer = row_converter.empty_rows(starting_rows_capacity, starting_data_capacity); + Ok(Self { schema, row_converter, @@ -103,6 +106,7 @@ impl GroupValuesRows { rows_buffer, random_state: Default::default(), block_size: None, + group_index_parse_fn: BlockedGroupIndex::new_flat, }) } } @@ -134,11 +138,7 @@ impl GroupValues for GroupValuesRows { batch_hashes.resize(n_rows, 0); create_hashes(cols, &self.random_state, batch_hashes)?; - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; + let group_index_parse_fn = self.group_index_parse_fn; for (row, &target_hash) in batch_hashes.iter().enumerate() { let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { @@ -286,11 +286,7 @@ impl GroupValues for GroupValuesRows { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; + let group_index_parse_fn = self.group_index_parse_fn; unsafe { for bucket in self.map.iter() { @@ -357,6 +353,11 @@ impl GroupValues for GroupValuesRows { self.map.clear(); self.group_values.clear(); self.block_size = block_size; + self.group_index_parse_fn = if block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; Ok(()) } From 3e409badd75aac89e6662deb08dddeab6d329db8 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 20:41:28 +0800 Subject: [PATCH 058/107] tmp. --- .../src/aggregate/groups_accumulator.rs | 2 -- .../aggregate/groups_accumulator/accumulate.rs | 11 +++++++++-- datafusion/functions-aggregate/src/count.rs | 16 +++++++++++++--- .../src/aggregates/group_values/row.rs | 13 +++++++++++-- 4 files changed, 33 insertions(+), 9 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index ca7b778bed90..282ccb1fe3fd 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -462,7 +462,6 @@ impl BlockedGroupIndex { } } - #[inline] pub fn new_flat(raw_index: usize) -> Self { Self { block_id: 0, @@ -471,7 +470,6 @@ impl BlockedGroupIndex { } } - #[inline] pub fn new_blocked(raw_index: usize) -> Self { let block_id = ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; 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 6cc72bd00877..1c87d6709cc9 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -430,11 +430,17 @@ impl BlockedNullState { ); let seen_values_blocks = &mut self.seen_values_blocks; + let group_index_parse_fn = if self.block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; + do_accumulate( group_indices, values, opt_filter, - self.group_index_parse_fn, + group_index_parse_fn, value_fn, |index: &BlockedGroupIndex| { seen_values_blocks[index.block_id].set_bit(index.block_offset, true); @@ -582,8 +588,9 @@ fn do_accumulate( T: ArrowPrimitiveType + Send, F1: FnMut(&BlockedGroupIndex, T::Native) + Send, F2: FnMut(&BlockedGroupIndex) + Send, -{ +{ let data: &[T::Native] = values.values(); + let group_index_parse_fn = |raw_index| { group_index_parse_fn(raw_index) }; match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index a87bbe27396e..24d8141afaac 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -402,7 +402,12 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - let group_index_parse_fn = self.group_index_parse_fn; + let group_index_parse_fn = if self.block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; + accumulate_indices( group_indices, values.logical_nulls().as_ref(), @@ -439,8 +444,13 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.block_size, 0, ); - - let group_index_parse_fn = self.group_index_parse_fn; + + let group_index_parse_fn = if self.block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; + do_count_merge_batch( values, group_indices, diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index f710af3214f0..25de38af4386 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -138,7 +138,11 @@ impl GroupValues for GroupValuesRows { batch_hashes.resize(n_rows, 0); create_hashes(cols, &self.random_state, batch_hashes)?; - let group_index_parse_fn = self.group_index_parse_fn; + let group_index_parse_fn = if self.block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; for (row, &target_hash) in batch_hashes.iter().enumerate() { let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { @@ -286,7 +290,12 @@ impl GroupValues for GroupValuesRows { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; - let group_index_parse_fn = self.group_index_parse_fn; + + let group_index_parse_fn = if self.block_size.is_some() { + BlockedGroupIndex::new_blocked + } else { + BlockedGroupIndex::new_flat + }; unsafe { for bucket in self.map.iter() { From 25269f4e68aaa2db46a7ec40adf9dd802d4350f3 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 21:37:23 +0800 Subject: [PATCH 059/107] tmp3 --- datafusion/functions-aggregate/src/count.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 24d8141afaac..7cd59957ec57 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -413,7 +413,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { values.logical_nulls().as_ref(), opt_filter, |group_index| { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = BlockedGroupIndex::new_flat(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += 1; @@ -456,7 +456,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { group_indices, opt_filter, |group_index, partial_count| { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = BlockedGroupIndex::new_flat(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += partial_count; From 4e2b9bc1713ddcb2cb4ca54b3627e4fdc12ba235 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 21:53:48 +0800 Subject: [PATCH 060/107] tmp4 --- .../src/aggregate/groups_accumulator.rs | 4 +++- .../src/aggregate/groups_accumulator/accumulate.rs | 1 - datafusion/functions-aggregate/src/count.rs | 4 ++-- datafusion/physical-plan/src/aggregates/group_values/row.rs | 4 ++-- 4 files changed, 7 insertions(+), 6 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 282ccb1fe3fd..a08cba3b1560 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -454,7 +454,7 @@ pub struct BlockedGroupIndex { impl BlockedGroupIndex { #[inline] - pub fn new(block_id: usize, block_offset: usize, is_blocked: bool) -> Self { + pub fn new_from_parts(block_id: usize, block_offset: usize, is_blocked: bool) -> Self { Self { block_id, block_offset, @@ -462,6 +462,7 @@ impl BlockedGroupIndex { } } + #[inline] pub fn new_flat(raw_index: usize) -> Self { Self { block_id: 0, @@ -470,6 +471,7 @@ impl BlockedGroupIndex { } } + #[inline] pub fn new_blocked(raw_index: usize) -> Self { let block_id = ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; 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 1c87d6709cc9..00c30ad3b166 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -590,7 +590,6 @@ fn do_accumulate( F2: FnMut(&BlockedGroupIndex) + Send, { let data: &[T::Native] = values.values(); - let group_index_parse_fn = |raw_index| { group_index_parse_fn(raw_index) }; match (values.null_count() > 0, opt_filter) { // no nulls, no filter, (false, None) => { diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 7cd59957ec57..24d8141afaac 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -413,7 +413,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { values.logical_nulls().as_ref(), opt_filter, |group_index| { - let blocked_index = BlockedGroupIndex::new_flat(group_index); + let blocked_index = group_index_parse_fn(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += 1; @@ -456,7 +456,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { group_indices, opt_filter, |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new_flat(group_index); + let blocked_index = group_index_parse_fn(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += partial_count; diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 25de38af4386..1c72bfc84c9e 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -186,7 +186,7 @@ impl GroupValues for GroupValuesRows { let blk_offset = cur_blk.num_rows(); cur_blk.push(group_rows.row(row)); - let blocked_index = BlockedGroupIndex::new( + let blocked_index = BlockedGroupIndex::new_from_parts( blk_id, blk_offset, self.block_size.is_some(), @@ -305,7 +305,7 @@ impl GroupValues for GroupValuesRows { match old_blk_idx.block_id.checked_sub(1) { // Group index was >= n, shift value down Some(new_blk_id) => { - let new_group_idx = BlockedGroupIndex::new( + let new_group_idx = BlockedGroupIndex::new_from_parts( new_blk_id, old_blk_idx.block_offset, true, From 07deb39c207413b547d65df3a39a647346da3fd6 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 22:04:25 +0800 Subject: [PATCH 061/107] tmp5. --- .../aggregate/groups_accumulator/accumulate.rs | 8 +------- datafusion/functions-aggregate/src/count.rs | 18 +++--------------- 2 files changed, 4 insertions(+), 22 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 00c30ad3b166..6e3f319a2d0f 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -430,17 +430,11 @@ impl BlockedNullState { ); let seen_values_blocks = &mut self.seen_values_blocks; - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - do_accumulate( group_indices, values, opt_filter, - group_index_parse_fn, + BlockedGroupIndex::new_flat, value_fn, |index: &BlockedGroupIndex| { seen_values_blocks[index.block_id].set_bit(index.block_offset, true); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 24d8141afaac..41b275dbe163 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -402,18 +402,12 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - accumulate_indices( group_indices, values.logical_nulls().as_ref(), opt_filter, |group_index| { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = BlockedGroupIndex::new_flat(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += 1; @@ -444,19 +438,13 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.block_size, 0, ); - - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - + do_count_merge_batch( values, group_indices, opt_filter, |group_index, partial_count| { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = BlockedGroupIndex::new_flat(group_index); let count = &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; *count += partial_count; From 5101165ff388bfbc39cf111ca57601f80bf943ea Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 22:16:54 +0800 Subject: [PATCH 062/107] just keep the if else. --- .../groups_accumulator/accumulate.rs | 44 ++++++++++++------- datafusion/functions-aggregate/src/count.rs | 38 +++++++++++----- 2 files changed, 54 insertions(+), 28 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 6e3f319a2d0f..ac89639ac0ce 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -25,9 +25,7 @@ use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; -use crate::aggregate::groups_accumulator::{ - BlockedGroupIndex, Blocks, -}; +use crate::aggregate::groups_accumulator::{BlockedGroupIndex, 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. /// @@ -430,16 +428,29 @@ impl BlockedNullState { ); let seen_values_blocks = &mut self.seen_values_blocks; - do_accumulate( - group_indices, - values, - opt_filter, - BlockedGroupIndex::new_flat, - value_fn, - |index: &BlockedGroupIndex| { - seen_values_blocks[index.block_id].set_bit(index.block_offset, true); - }, - ); + if self.block_size.is_some() { + do_accumulate( + group_indices, + values, + opt_filter, + BlockedGroupIndex::new_blocked, + value_fn, + |index: &BlockedGroupIndex| { + seen_values_blocks[index.block_id].set_bit(index.block_offset, true); + }, + ) + } else { + do_accumulate( + group_indices, + values, + opt_filter, + BlockedGroupIndex::new_flat, + value_fn, + |index: &BlockedGroupIndex| { + seen_values_blocks[index.block_id].set_bit(index.block_offset, true); + }, + ); + } } /// Similar as [NullState::build] but support the blocked version accumulator @@ -571,18 +582,19 @@ pub fn accumulate_indices( } } -fn do_accumulate( +fn do_accumulate( group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, - group_index_parse_fn: fn(usize) -> BlockedGroupIndex, + group_index_parse_fn: G, mut value_fn: F1, mut set_valid_fn: F2, ) where T: ArrowPrimitiveType + Send, + G: Fn(usize) -> BlockedGroupIndex, F1: FnMut(&BlockedGroupIndex, T::Native) + Send, F2: FnMut(&BlockedGroupIndex) + Send, -{ +{ let data: &[T::Native] = values.values(); match (values.null_count() > 0, opt_filter) { // no nulls, no filter, diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 41b275dbe163..4583191441ec 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -438,18 +438,32 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.block_size, 0, ); - - do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = - &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; - *count += partial_count; - }, - ); + + if self.block_size.is_some() { + do_count_merge_batch( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let blocked_index = BlockedGroupIndex::new_blocked(group_index); + let count = + &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + *count += partial_count; + }, + ); + } else { + do_count_merge_batch( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let blocked_index = BlockedGroupIndex::new_flat(group_index); + let count = + &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + *count += partial_count; + }, + ); + } Ok(()) } From e10a4bb2bdb49223ee2ff185e9a469735556799e Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 22:21:51 +0800 Subject: [PATCH 063/107] remove some unnecessary codes. --- .../src/aggregate/groups_accumulator.rs | 6 +- .../groups_accumulator/accumulate.rs | 3 - .../aggregate/groups_accumulator/prim_op.rs | 7 +-- datafusion/functions-aggregate/src/count.rs | 56 ++++++++++--------- .../src/aggregates/group_values/row.rs | 8 --- 5 files changed, 39 insertions(+), 41 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index a08cba3b1560..d7b6f6a2a642 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -454,7 +454,11 @@ pub struct BlockedGroupIndex { impl BlockedGroupIndex { #[inline] - pub fn new_from_parts(block_id: usize, block_offset: usize, is_blocked: bool) -> Self { + pub fn new_from_parts( + block_id: usize, + block_offset: usize, + is_blocked: bool, + ) -> Self { Self { block_id, block_offset, 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 ac89639ac0ce..1b2be4592867 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -369,8 +369,6 @@ pub struct BlockedNullState { seen_values_blocks: Blocks, block_size: Option, - - group_index_parse_fn: fn(usize) -> BlockedGroupIndex, } impl Default for BlockedNullState { @@ -390,7 +388,6 @@ impl BlockedNullState { Self { seen_values_blocks: Blocks::new(), block_size, - group_index_parse_fn, } } 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 370e4df6eb80..5f1b9ae2c396 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 @@ -27,7 +27,7 @@ use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; use crate::aggregate::groups_accumulator::accumulate::BlockedNullState; use crate::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, Blocks, EmitToExt, VecBlocks + ensure_enough_room_for_values, Blocks, EmitToExt, VecBlocks, }; /// An accumulator that implements a single operation over @@ -112,15 +112,14 @@ where self.block_size, self.starting_value, ); - + self.null_state.accumulate( group_indices, values, opt_filter, total_num_groups, |index, new_value| { - let value = &mut self.values_blocks[index.block_id] - [index.block_offset]; + let value = &mut self.values_blocks[index.block_id][index.block_offset]; (self.prim_fn)(value, new_value); }, ); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 4583191441ec..6be06acf8a16 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -18,7 +18,7 @@ use ahash::RandomState; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks + ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, }; use std::collections::HashSet; use std::ops::BitAnd; @@ -364,8 +364,6 @@ struct CountGroupsAccumulator { counts: VecBlocks, block_size: Option, - - group_index_parse_fn: fn(usize) -> BlockedGroupIndex, } impl CountGroupsAccumulator { @@ -373,7 +371,6 @@ impl CountGroupsAccumulator { Self { counts: Blocks::new(), block_size: None, - group_index_parse_fn: BlockedGroupIndex::new_flat, } } } @@ -402,17 +399,31 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = - &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; - *count += 1; - }, - ); + if self.block_size.is_some() { + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + let blocked_index = BlockedGroupIndex::new_blocked(group_index); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + *count += 1; + }, + ); + } else { + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + let blocked_index = BlockedGroupIndex::new_flat(group_index); + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; + *count += 1; + }, + ); + } Ok(()) } @@ -438,7 +449,7 @@ impl GroupsAccumulator for CountGroupsAccumulator { self.block_size, 0, ); - + if self.block_size.is_some() { do_count_merge_batch( values, @@ -446,8 +457,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter, |group_index, partial_count| { let blocked_index = BlockedGroupIndex::new_blocked(group_index); - let count = - &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; *count += partial_count; }, ); @@ -458,8 +469,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter, |group_index, partial_count| { let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = - &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id] + [blocked_index.block_offset]; *count += partial_count; }, ); @@ -561,11 +572,6 @@ impl GroupsAccumulator for CountGroupsAccumulator { fn alter_block_size(&mut self, block_size: Option) -> Result<()> { self.counts.clear(); self.block_size = block_size; - self.group_index_parse_fn = if block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 1c72bfc84c9e..820a3892f7da 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -75,8 +75,6 @@ pub struct GroupValuesRows { /// Mode about current GroupValuesRows block_size: Option, - - group_index_parse_fn: fn(usize) -> BlockedGroupIndex, } impl GroupValuesRows { @@ -106,7 +104,6 @@ impl GroupValuesRows { rows_buffer, random_state: Default::default(), block_size: None, - group_index_parse_fn: BlockedGroupIndex::new_flat, }) } } @@ -362,11 +359,6 @@ impl GroupValues for GroupValuesRows { self.map.clear(); self.group_values.clear(); self.block_size = block_size; - self.group_index_parse_fn = if block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; Ok(()) } From cc117d4a2f222c437aa397a0ae0c11b1a7a90918 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 23:29:56 +0800 Subject: [PATCH 064/107] adapt the new great comments. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 5390c8a9a0bb..f5cad5897036 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -430,6 +430,9 @@ pub(crate) struct GroupedHashAggregateStream { /// current stream. skip_aggregation_probe: Option, + /// Have we enabled the blocked optimization for group values and accumulators. + enable_blocked_group_states: bool, + // ======================================================================== // EXECUTION RESOURCES: // Fields related to managing execution resources and monitoring performance. @@ -442,8 +445,6 @@ pub(crate) struct GroupedHashAggregateStream { /// The [`RuntimeEnv`] associated with the [`TaskContext`] argument runtime: Arc, - /// Have we enabled the blocked optimization for group values and accumulators. - enable_blocked_group_states: bool, } impl GroupedHashAggregateStream { From c65b80823ec7c7529a726f6a4570dc5d006c765d Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 25 Aug 2024 23:57:35 +0800 Subject: [PATCH 065/107] add option and disable the blocked optimization by default. --- datafusion/common/src/config.rs | 13 +++++++++++++ datafusion/physical-plan/src/aggregates/row_hash.rs | 8 +++++++- 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 37d26c6f00c4..72eafd5a503e 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -341,6 +341,19 @@ config_namespace! { /// if the source of statistics is accurate. /// We plan to make this the default in the future. pub use_row_number_estimates_to_optimize_partitioning: bool, default = false + + /// Should DataFusion use the the blocked approach to manage the groups + /// values and their related states in accumulators. By default, the single + /// approach will be used, and such group values and states will be managed + /// using a single big block(can think a `Vec`), obviously as the block growing up, + /// many copies will be triggered and finally get a bad performance. + /// If setting this flag to `true`, the blocked approach will be used. + /// We will allocate the `block size` capacity for block first, and when we + /// found the block has been filled to `block size` limit, we will allocate + /// next block rather than growing current block and copying the data. This + /// approach can eliminate all unnecessary copies and get a good performance finally. + /// We plan to make this the default in the future when tests are enough. + pub enable_aggregation_group_states_blocked_approach: bool, default = false } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index f5cad5897036..abac46acd991 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -601,6 +601,7 @@ impl GroupedHashAggregateStream { /// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. /// The blocked optimization will be enabled when: +/// - When `enable_aggregation_group_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) @@ -613,7 +614,12 @@ fn maybe_enable_blocked_group_states( block_size: usize, group_ordering: &GroupOrdering, ) -> Result { - if !matches!(group_ordering, GroupOrdering::None) + if !context + .session_config() + .options() + .execution + .enable_aggregation_group_states_blocked_approach + || !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() || enable_spilling(context.memory_pool().as_ref()) { From 0a26de8cc04045c3b19b782f7e13483558fe7951 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 00:10:23 +0800 Subject: [PATCH 066/107] remove codes about outdated `switch_to_mdoe`. --- datafusion/common/src/config.rs | 4 +- .../expr-common/src/groups_accumulator.rs | 42 ++++--------------- .../groups_accumulator/accumulate.rs | 6 --- .../src/aggregates/group_values/mod.rs | 25 +++++------ .../physical-plan/src/aggregates/row_hash.rs | 3 +- 5 files changed, 23 insertions(+), 57 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 72eafd5a503e..ee5cc2695a54 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -343,13 +343,13 @@ config_namespace! { pub use_row_number_estimates_to_optimize_partitioning: bool, default = false /// Should DataFusion use the the blocked approach to manage the groups - /// values and their related states in accumulators. By default, the single + /// values and their related states in accumulators. By default, the single /// approach will be used, and such group values and states will be managed /// using a single big block(can think a `Vec`), obviously as the block growing up, /// many copies will be triggered and finally get a bad performance. /// If setting this flag to `true`, the blocked approach will be used. /// We will allocate the `block size` capacity for block first, and when we - /// found the block has been filled to `block size` limit, we will allocate + /// found the block has been filled to `block size` limit, we will allocate /// next block rather than growing current block and copying the data. This /// approach can eliminate all unnecessary copies and get a good performance finally. /// We plan to make this the default in the future when tests are enough. diff --git a/datafusion/expr-common/src/groups_accumulator.rs b/datafusion/expr-common/src/groups_accumulator.rs index f0463ae58112..9610809c601e 100644 --- a/datafusion/expr-common/src/groups_accumulator.rs +++ b/datafusion/expr-common/src/groups_accumulator.rs @@ -41,23 +41,6 @@ pub enum EmitTo { NextBlock(bool), } -/// Mode for `accumulators` and `group values` -/// -/// Their meanings: -/// - Flat, the values in them will be managed with a single `Vec`. -/// It will grow constantly when more and more values are inserted, -/// that leads to a considerable amount of copying, and finally a bad performance. -/// -/// - Blocked(block_size), the values in them will be managed with multiple `Vec`s. -/// When the block is large enough(reach block_size), a new block will be allocated -/// and used for inserting. -/// Obviously, this strategy can avoid copying and get a good performance. -#[derive(Debug, Clone, Copy)] -pub enum GroupStatesMode { - Flat, - Blocked(usize), -} - /// `GroupAccumulator` implements a single aggregate (e.g. AVG) and /// stores the state for *all* groups internally. /// @@ -150,24 +133,17 @@ pub trait GroupsAccumulator: Send { false } - /// Switch the accumulator to flat or blocked mode. - /// You can see detail about the mode on [GroupStatesMode]. + /// Alter the block size in the accumulator /// - /// After switching mode, all data in previous mode will be cleared. - fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { - if matches!(&mode, GroupStatesMode::Blocked(_)) { - return Err(DataFusionError::NotImplemented( - "this accumulator doesn't support blocked mode yet".to_string(), - )); - } - - Ok(()) - } - - /// Switch the accumulator to flat or blocked mode. - /// You can see detail about the mode on [GroupStatesMode]. + /// 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. /// - /// After switching mode, all data in previous mode will be cleared. fn alter_block_size(&mut self, block_size: Option) -> Result<()> { if block_size.is_some() { return Err(DataFusionError::NotImplemented( 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 1b2be4592867..3ec0d9eda4e0 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -379,12 +379,6 @@ impl Default for BlockedNullState { impl BlockedNullState { pub fn new(block_size: Option) -> Self { - let group_index_parse_fn = if block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; - Self { seen_values_blocks: Blocks::new(), block_size, diff --git a/datafusion/physical-plan/src/aggregates/group_values/mod.rs b/datafusion/physical-plan/src/aggregates/group_values/mod.rs index f93959aa8ba6..a7f86f3ba23f 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/mod.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/mod.rs @@ -22,7 +22,7 @@ use bytes_view::GroupValuesBytesView; use datafusion_common::{DataFusionError, Result}; pub(crate) mod primitive; -use datafusion_expr::{groups_accumulator::GroupStatesMode, EmitTo}; +use datafusion_expr::EmitTo; use primitive::GroupValuesPrimitive; mod row; @@ -58,20 +58,17 @@ pub trait GroupValues: Send { false } - /// Switch the group values to flat or blocked mode. - /// You can see detail about the mode on [GroupStatesMode]. + /// 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. /// - /// After switching mode, all data in previous mode will be cleared. - fn switch_to_mode(&mut self, mode: GroupStatesMode) -> Result<()> { - if matches!(&mode, GroupStatesMode::Blocked(_)) { - return Err(DataFusionError::NotImplemented( - "this group values doesn't support blocked mode yet".to_string(), - )); - } - - Ok(()) - } - fn alter_block_size(&mut self, block_size: Option) -> Result<()> { if block_size.is_some() { return Err(DataFusionError::NotImplemented( diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index abac46acd991..bcff005a383b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -45,7 +45,6 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_expr::groups_accumulator::GroupStatesMode; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ @@ -1104,7 +1103,7 @@ impl GroupedHashAggregateStream { // because the blocked mode can't support `Emit::First(exact n)` which is needed in // streaming aggregation. if self.enable_blocked_group_states { - self.group_values.switch_to_mode(GroupStatesMode::Flat)?; + self.group_values.alter_block_size(None)?; self.accumulators .iter_mut() .try_for_each(|acc| acc.alter_block_size(None))?; From 96f8be8bc84f0dcd0d137072438736c3da2c90f7 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 00:19:41 +0800 Subject: [PATCH 067/107] fix test. --- .../groups_accumulator/accumulate.rs | 1420 ++++++++--------- 1 file changed, 709 insertions(+), 711 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 3ec0d9eda4e0..4af736084361 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -808,715 +808,713 @@ fn ensure_enough_room_for_blocked_nulls( #[cfg(test)] mod test { - // use super::*; - - // use arrow::array::UInt32Array; - // use rand::{rngs::ThreadRng, Rng}; - // use std::collections::HashSet; - - // /// Null state's behaviors needed in accumulate test - // trait TestNullState { - // fn accumulate( - // &mut self, - // group_indices: &[usize], - // values: &PrimitiveArray, - // opt_filter: Option<&BooleanArray>, - // total_num_groups: usize, - // accumulated_values: &mut Vec<(usize, T::Native)>, - // ) where - // T: ArrowPrimitiveType + Send; - - // fn build_bool_buffer(&self) -> BooleanBuffer; - - // fn build_null_buffer(&mut self) -> NullBuffer; - // } - - // /// The original `NullState` - // impl TestNullState for NullState { - // fn accumulate( - // &mut self, - // group_indices: &[usize], - // values: &PrimitiveArray, - // opt_filter: Option<&BooleanArray>, - // total_num_groups: usize, - // accumulated_values: &mut Vec<(usize, T::Native)>, - // ) where - // T: ArrowPrimitiveType + Send, - // { - // self.accumulate( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // |group_index, value| { - // accumulated_values.push((group_index, value)); - // }, - // ); - // } - - // fn build_bool_buffer(&self) -> BooleanBuffer { - // self.seen_values.finish_cloned() - // } - - // fn build_null_buffer(&mut self) -> NullBuffer { - // self.build(EmitTo::All) - // } - // } - - // /// The new `BlockedNullState` in flat mode - // struct BlockedNullStateInFlatMode(BlockedNullState); - - // impl BlockedNullStateInFlatMode { - // fn new() -> Self { - // let null_state = BlockedNullState::new(GroupStatesMode::Flat); - - // Self(null_state) - // } - // } - - // impl TestNullState for BlockedNullStateInFlatMode { - // fn accumulate( - // &mut self, - // group_indices: &[usize], - // values: &PrimitiveArray, - // opt_filter: Option<&BooleanArray>, - // total_num_groups: usize, - // accumulated_values: &mut Vec<(usize, T::Native)>, - // ) where - // T: ArrowPrimitiveType + Send, - // { - // self.0.accumulate_for_flat( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // |group_index, value| { - // accumulated_values.push((group_index, value)); - // }, - // ); - // } - - // fn build_bool_buffer(&self) -> BooleanBuffer { - // self.0.seen_values_blocks.current().unwrap().finish_cloned() - // } - - // fn build_null_buffer(&mut self) -> NullBuffer { - // self.0.build(EmitTo::All) - // } - // } - - // /// The new `BlockedNullState` in blocked mode - // struct BlockedNullStateInBlockedMode { - // null_state: BlockedNullState, - // block_size: usize, - // } - - // impl BlockedNullStateInBlockedMode { - // fn new() -> Self { - // let null_state = BlockedNullState::new(GroupStatesMode::Blocked(4)); - - // Self { - // null_state, - // block_size: 4, - // } - // } - // } - - // impl TestNullState for BlockedNullStateInBlockedMode { - // fn accumulate( - // &mut self, - // group_indices: &[usize], - // values: &PrimitiveArray, - // opt_filter: Option<&BooleanArray>, - // total_num_groups: usize, - // accumulated_values: &mut Vec<(usize, T::Native)>, - // ) where - // T: ArrowPrimitiveType + Send, - // { - // // Convert group indices to blocked style - // let blocked_indices = group_indices - // .iter() - // .map(|idx| { - // let block_id = *idx / self.block_size; - // let block_offset = *idx % self.block_size; - // BlockedGroupIndex::new_from_parts(block_id, block_offset) - // .as_packed_index() - // }) - // .collect::>(); - - // self.null_state.accumulate_for_blocked( - // &blocked_indices, - // values, - // opt_filter, - // total_num_groups, - // self.block_size, - // |group_index, value| { - // let blocked_index = BlockedGroupIndex::new(group_index); - // let flat_index = blocked_index.block_id * self.block_size - // + blocked_index.block_offset; - // accumulated_values.push((flat_index, value)); - // }, - // ); - // } - - // fn build_bool_buffer(&self) -> BooleanBuffer { - // let mut ret_builder = BooleanBufferBuilder::new(0); - // for blk in self.null_state.seen_values_blocks.iter() { - // let buf = blk.finish_cloned(); - // for seen in buf.iter() { - // ret_builder.append(seen); - // } - // } - // ret_builder.finish() - // } - - // fn build_null_buffer(&mut self) -> NullBuffer { - // let mut ret_builder = BooleanBufferBuilder::new(0); - // loop { - // let blk = self.null_state.build(EmitTo::NextBlock(false)); - // if blk.is_empty() { - // break; - // } - - // for seen in blk.iter() { - // ret_builder.append(seen); - // } - // } - - // NullBuffer::new(ret_builder.finish()) - // } - // } - - // /// Accumulate test mode - // /// - Original, test the original `NullState` - // /// - Flat, test the `BlockedNullState` in flat mode - // /// - Blocked, test the `BlockedNullState` in blocked mode - // #[derive(Debug, Clone, Copy)] - // enum AccumulateTest { - // Original, - // Flat, - // Blocked, - // } - - // impl AccumulateTest { - // fn run( - // &self, - // group_indices: &[usize], - // values: &UInt32Array, - // opt_filter: Option<&BooleanArray>, - // total_num_groups: usize, - // ) { - // match self { - // AccumulateTest::Original => { - // Fixture::accumulate_test( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // NullState::new(), - // ); - // } - // AccumulateTest::Flat => { - // Fixture::accumulate_test( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // BlockedNullStateInFlatMode::new(), - // ); - // } - // AccumulateTest::Blocked => { - // Fixture::accumulate_test( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // BlockedNullStateInBlockedMode::new(), - // ); - // } - // } - // } - // } - - // #[test] - // fn accumulate_test_original() { - // do_accumulate_test(AccumulateTest::Original); - // } - - // #[test] - // fn accumulate_test_flat() { - // do_accumulate_test(AccumulateTest::Flat); - // } - - // #[test] - // fn accumulate_test_blocked() { - // do_accumulate_test(AccumulateTest::Blocked); - // } - - // #[test] - // fn accumulate_fuzz_test_original() { - // do_accumulate_fuzz_test(AccumulateTest::Original); - // } - - // #[test] - // fn accumulate_fuzz_test_flat() { - // do_accumulate_fuzz_test(AccumulateTest::Flat); - // } - - // #[test] - // fn accumulate_fuzz_test_blocked() { - // do_accumulate_fuzz_test(AccumulateTest::Blocked); - // } - - // fn do_accumulate_test(accumulate_test: AccumulateTest) { - // let group_indices = (0..100).collect(); - // let values = (0..100).map(|i| (i + 1) * 10).collect(); - // let values_with_nulls = (0..100) - // .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) - // .collect(); - - // // default to every fifth value being false, every even - // // being null - // let filter: BooleanArray = (0..100) - // .map(|i| { - // let is_even = i % 2 == 0; - // let is_fifth = i % 5 == 0; - // if is_even { - // None - // } else if is_fifth { - // Some(false) - // } else { - // Some(true) - // } - // }) - // .collect(); - - // Fixture { - // group_indices, - // values, - // values_with_nulls, - // filter, - // accumulate_test, - // } - // .run() - // } - - // fn do_accumulate_fuzz_test(accumulate_test: AccumulateTest) { - // let mut rng = rand::thread_rng(); - // for _ in 0..100 { - // Fixture::new_random(&mut rng, accumulate_test).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, - - // /// tested null state for value test - // accumulate_test: AccumulateTest, - // } - - // impl Fixture { - // fn new_random(rng: &mut ThreadRng, accumulate_test: AccumulateTest) -> 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(); - - // // 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, - // accumulate_test, - // } - // } - - // /// 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.run( - // group_indices, - // &values_array, - // None, - // total_num_groups, - // ); - - // // nulls, no filters - // self.accumulate_test.run( - // group_indices, - // &values_with_nulls_array, - // None, - // total_num_groups, - // ); - - // // no nulls, filters - // self.accumulate_test.run( - // group_indices, - // &values_array, - // Some(filter), - // total_num_groups, - // ); - - // // nulls, filters - // self.accumulate_test.run( - // group_indices, - // &values_with_nulls_array, - // Some(filter), - // total_num_groups, - // ); - // } - - // /// 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, - // null_state_for_value_test: S, - // ) { - // Self::accumulate_values_test( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // null_state_for_value_test, - // ); - // 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(); - - // // TODO: test the `BlockedNullState` after supporting `accumulate_boolean` in it - // Self::accumulate_boolean_test( - // group_indices, - // &boolean_values, - // opt_filter, - // total_num_groups, - // ); - // } - - // /// 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, - // mut null_state: S, - // ) { - // let mut accumulated_values = vec![]; - - // null_state.accumulate( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // &mut accumulated_values, - // ); - - // // 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_bool_buffer(); - // 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_null_buffer(); - - // 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, - // ) { - // let mut accumulated_values = vec![]; - // let mut null_state = NullState::new(); - - // null_state.accumulate_boolean( - // group_indices, - // values, - // opt_filter, - // total_num_groups, - // |group_index, value| { - // accumulated_values.push((group_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.seen_values.finish_cloned(); - // 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); - - // 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_ensure_room_for_blocked_nulls() { - // let mut blocks: Blocks = Blocks::new(); - // let block_size = 4; - - // // 0 total_num_groups, should be no blocks - // ensure_enough_room_for_blocked_nulls(&mut blocks, 0, block_size, false); - // assert_eq!(blocks.num_blocks(), 0); - // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - // assert_eq!(total_len, 0); - - // // 0 -> 3 total_num_groups, blocks should look like: - // // [d, d, d, empty] - // ensure_enough_room_for_blocked_nulls(&mut blocks, 3, block_size, false); - // assert_eq!(blocks.num_blocks(), 1); - // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - // assert_eq!(total_len, 3); - - // // 3 -> 8 total_num_groups, blocks should look like: - // // [d, d, d, d], [d, d, d, d] - // ensure_enough_room_for_blocked_nulls(&mut blocks, 8, block_size, false); - // assert_eq!(blocks.num_blocks(), 2); - // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - // assert_eq!(total_len, 8); - - // // 8 -> 13 total_num_groups, blocks should look like: - // // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] - // ensure_enough_room_for_blocked_nulls(&mut blocks, 13, block_size, false); - // assert_eq!(blocks.num_blocks(), 4); - // let total_len = blocks.iter().map(|blk| blk.len()).sum::(); - // assert_eq!(total_len, 13); - // } + use super::*; + + use arrow::array::UInt32Array; + use rand::{rngs::ThreadRng, Rng}; + use std::collections::HashSet; + + /// Null state's behaviors needed in accumulate test + trait TestNullState { + fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + accumulated_values: &mut Vec<(usize, T::Native)>, + ) where + T: ArrowPrimitiveType + Send; + + fn build_bool_buffer(&self) -> BooleanBuffer; + + fn build_null_buffer(&mut self) -> NullBuffer; + } + + /// The original `NullState` + impl TestNullState for NullState { + fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + accumulated_values: &mut Vec<(usize, T::Native)>, + ) where + T: ArrowPrimitiveType + Send, + { + self.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_index, value)); + }, + ); + } + + fn build_bool_buffer(&self) -> BooleanBuffer { + self.seen_values.finish_cloned() + } + + fn build_null_buffer(&mut self) -> NullBuffer { + self.build(EmitTo::All) + } + } + + /// The new `BlockedNullState` in flat mode + struct BlockedNullStateInFlatMode(BlockedNullState); + + impl BlockedNullStateInFlatMode { + fn new() -> Self { + let null_state = BlockedNullState::new(None); + + Self(null_state) + } + } + + impl TestNullState for BlockedNullStateInFlatMode { + fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + accumulated_values: &mut Vec<(usize, T::Native)>, + ) where + T: ArrowPrimitiveType + Send, + { + self.0.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_index.as_packed_index(), value)); + }, + ); + } + + fn build_bool_buffer(&self) -> BooleanBuffer { + self.0.seen_values_blocks.current().unwrap().finish_cloned() + } + + fn build_null_buffer(&mut self) -> NullBuffer { + self.0.build(EmitTo::All) + } + } + + /// The new `BlockedNullState` in blocked mode + struct BlockedNullStateInBlockedMode { + null_state: BlockedNullState, + block_size: usize, + } + + impl BlockedNullStateInBlockedMode { + fn new() -> Self { + let null_state = BlockedNullState::new(Some(4)); + + Self { + null_state, + block_size: 4, + } + } + } + + impl TestNullState for BlockedNullStateInBlockedMode { + fn accumulate( + &mut self, + group_indices: &[usize], + values: &PrimitiveArray, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + accumulated_values: &mut Vec<(usize, T::Native)>, + ) where + T: ArrowPrimitiveType + Send, + { + // Convert group indices to blocked style + let blocked_indices = group_indices + .iter() + .map(|idx| { + let block_id = *idx / self.block_size; + let block_offset = *idx % self.block_size; + BlockedGroupIndex::new_from_parts(block_id, block_offset, true) + .as_packed_index() + }) + .collect::>(); + + self.null_state.accumulate( + &blocked_indices, + values, + opt_filter, + total_num_groups, + |blocked_index, value| { + let flat_index = blocked_index.block_id * self.block_size + + blocked_index.block_offset; + accumulated_values.push((flat_index, value)); + }, + ); + } + + fn build_bool_buffer(&self) -> BooleanBuffer { + let mut ret_builder = BooleanBufferBuilder::new(0); + for blk in self.null_state.seen_values_blocks.iter() { + let buf = blk.finish_cloned(); + for seen in buf.iter() { + ret_builder.append(seen); + } + } + ret_builder.finish() + } + + fn build_null_buffer(&mut self) -> NullBuffer { + let mut ret_builder = BooleanBufferBuilder::new(0); + loop { + let blk = self.null_state.build(EmitTo::NextBlock(false)); + if blk.is_empty() { + break; + } + + for seen in blk.iter() { + ret_builder.append(seen); + } + } + + NullBuffer::new(ret_builder.finish()) + } + } + + /// Accumulate test mode + /// - Original, test the original `NullState` + /// - Flat, test the `BlockedNullState` in flat mode + /// - Blocked, test the `BlockedNullState` in blocked mode + #[derive(Debug, Clone, Copy)] + enum AccumulateTest { + Original, + Flat, + Blocked, + } + + impl AccumulateTest { + fn run( + &self, + group_indices: &[usize], + values: &UInt32Array, + opt_filter: Option<&BooleanArray>, + total_num_groups: usize, + ) { + match self { + AccumulateTest::Original => { + Fixture::accumulate_test( + group_indices, + values, + opt_filter, + total_num_groups, + NullState::new(), + ); + } + AccumulateTest::Flat => { + Fixture::accumulate_test( + group_indices, + values, + opt_filter, + total_num_groups, + BlockedNullStateInFlatMode::new(), + ); + } + AccumulateTest::Blocked => { + Fixture::accumulate_test( + group_indices, + values, + opt_filter, + total_num_groups, + BlockedNullStateInBlockedMode::new(), + ); + } + } + } + } + + #[test] + fn accumulate_test_original() { + do_accumulate_test(AccumulateTest::Original); + } + + #[test] + fn accumulate_test_flat() { + do_accumulate_test(AccumulateTest::Flat); + } + + #[test] + fn accumulate_test_blocked() { + do_accumulate_test(AccumulateTest::Blocked); + } + + #[test] + fn accumulate_fuzz_test_original() { + do_accumulate_fuzz_test(AccumulateTest::Original); + } + + #[test] + fn accumulate_fuzz_test_flat() { + do_accumulate_fuzz_test(AccumulateTest::Flat); + } + + #[test] + fn accumulate_fuzz_test_blocked() { + do_accumulate_fuzz_test(AccumulateTest::Blocked); + } + + fn do_accumulate_test(accumulate_test: AccumulateTest) { + let group_indices = (0..100).collect(); + let values = (0..100).map(|i| (i + 1) * 10).collect(); + let values_with_nulls = (0..100) + .map(|i| if i % 3 == 0 { None } else { Some((i + 1) * 10) }) + .collect(); + + // default to every fifth value being false, every even + // being null + let filter: BooleanArray = (0..100) + .map(|i| { + let is_even = i % 2 == 0; + let is_fifth = i % 5 == 0; + if is_even { + None + } else if is_fifth { + Some(false) + } else { + Some(true) + } + }) + .collect(); + + Fixture { + group_indices, + values, + values_with_nulls, + filter, + accumulate_test, + } + .run() + } + + fn do_accumulate_fuzz_test(accumulate_test: AccumulateTest) { + let mut rng = rand::thread_rng(); + for _ in 0..100 { + Fixture::new_random(&mut rng, accumulate_test).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, + + /// tested null state for value test + accumulate_test: AccumulateTest, + } + + impl Fixture { + fn new_random(rng: &mut ThreadRng, accumulate_test: AccumulateTest) -> 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(); + + // 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, + accumulate_test, + } + } + + /// 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.run( + group_indices, + &values_array, + None, + total_num_groups, + ); + + // nulls, no filters + self.accumulate_test.run( + group_indices, + &values_with_nulls_array, + None, + total_num_groups, + ); + + // no nulls, filters + self.accumulate_test.run( + group_indices, + &values_array, + Some(filter), + total_num_groups, + ); + + // nulls, filters + self.accumulate_test.run( + group_indices, + &values_with_nulls_array, + Some(filter), + total_num_groups, + ); + } + + /// 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, + null_state_for_value_test: S, + ) { + Self::accumulate_values_test( + group_indices, + values, + opt_filter, + total_num_groups, + null_state_for_value_test, + ); + 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(); + + // TODO: test the `BlockedNullState` after supporting `accumulate_boolean` in it + Self::accumulate_boolean_test( + group_indices, + &boolean_values, + opt_filter, + total_num_groups, + ); + } + + /// 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, + mut null_state: S, + ) { + let mut accumulated_values = vec![]; + + null_state.accumulate( + group_indices, + values, + opt_filter, + total_num_groups, + &mut accumulated_values, + ); + + // 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_bool_buffer(); + 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_null_buffer(); + + 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, + ) { + let mut accumulated_values = vec![]; + let mut null_state = NullState::new(); + + null_state.accumulate_boolean( + group_indices, + values, + opt_filter, + total_num_groups, + |group_index, value| { + accumulated_values.push((group_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.seen_values.finish_cloned(); + 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); + + 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_ensure_room_for_blocked_nulls() { + let mut blocks: Blocks = Blocks::new(); + let block_size = 4; + + // 0 total_num_groups, should be no blocks + ensure_enough_room_for_blocked_nulls(&mut blocks, 0, block_size, false); + assert_eq!(blocks.num_blocks(), 0); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 0); + + // 0 -> 3 total_num_groups, blocks should look like: + // [d, d, d, empty] + ensure_enough_room_for_blocked_nulls(&mut blocks, 3, block_size, false); + assert_eq!(blocks.num_blocks(), 1); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 3); + + // 3 -> 8 total_num_groups, blocks should look like: + // [d, d, d, d], [d, d, d, d] + ensure_enough_room_for_blocked_nulls(&mut blocks, 8, block_size, false); + assert_eq!(blocks.num_blocks(), 2); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 8); + + // 8 -> 13 total_num_groups, blocks should look like: + // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] + ensure_enough_room_for_blocked_nulls(&mut blocks, 13, block_size, false); + assert_eq!(blocks.num_blocks(), 4); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 13); + } } From 6535b938301ef7002d4f15a6533cbd683ddb058f Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 00:23:25 +0800 Subject: [PATCH 068/107] fix clippy. --- .../src/aggregate/groups_accumulator.rs | 1 - datafusion/physical-plan/src/aggregates/group_values/row.rs | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index d7b6f6a2a642..ee24379042b3 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -29,7 +29,6 @@ use std::{ fmt::{self, Debug}, iter, ops::{Index, IndexMut}, - usize, }; use arrow::{ diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 820a3892f7da..953a10dde2af 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -239,7 +239,7 @@ impl GroupValues for GroupValuesRows { 0 }; - group_len = group_len + self.group_values.current().unwrap().num_rows(); + group_len += self.group_values.current().unwrap().num_rows(); group_len } From 781d00c3a3ce2c5c351c3f06904245e997340abc Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 00:35:50 +0800 Subject: [PATCH 069/107] try to eliminate more dynamic dispatch. --- .../src/aggregates/group_values/row.rs | 130 +++++++++--------- 1 file changed, 68 insertions(+), 62 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 953a10dde2af..9cdd1b1f4c86 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -135,73 +135,79 @@ impl GroupValues for GroupValuesRows { batch_hashes.resize(n_rows, 0); create_hashes(cols, &self.random_state, batch_hashes)?; - let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked - } else { - BlockedGroupIndex::new_flat - }; + let mut get_or_create_groups = + |group_index_parse_fn: fn(usize) -> BlockedGroupIndex| { + for (row, &target_hash) in batch_hashes.iter().enumerate() { + let entry = + self.map.get_mut(target_hash, |(exist_hash, group_idx)| { + // Somewhat surprisingly, this closure can be called even if the + // hash doesn't match, so check the hash first with an integer + // comparison first avoid the more expensive comparison with + // group value. https://github.com/apache/datafusion/pull/11718 + if target_hash != *exist_hash { + return false; + } - for (row, &target_hash) in batch_hashes.iter().enumerate() { - let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { - // Somewhat surprisingly, this closure can be called even if the - // hash doesn't match, so check the hash first with an integer - // comparison first avoid the more expensive comparison with - // group value. https://github.com/apache/datafusion/pull/11718 - if target_hash != *exist_hash { - return false; - } + // verify that the group that we are inserting with hash is + // actually the same key value as the group in + // existing_idx (aka group_values @ row) + let blocked_index = group_index_parse_fn(*group_idx); + group_rows.row(row) + == group_values[blocked_index.block_id] + .row(blocked_index.block_offset) + }); + + let group_idx = match entry { + // Existing group_index for this group value + Some((_hash, group_idx)) => *group_idx, + // 1.2 Need to create new entry for the group + None => { + // Add new entry to aggr_state and save newly created index + if let Some(blk_size) = self.block_size { + if group_values.current().unwrap().num_rows() == blk_size + { + // Use blk_size as offset cap, + // and old block's buffer size as buffer cap + let new_buf_cap = + rows_buffer_size(group_values.current().unwrap()); + let new_blk = self + .row_converter + .empty_rows(blk_size, new_buf_cap); + group_values.push_block(new_blk); + } + } - // verify that the group that we are inserting with hash is - // actually the same key value as the group in - // existing_idx (aka group_values @ row) - let blocked_index = group_index_parse_fn(*group_idx); - group_rows.row(row) - == group_values[blocked_index.block_id] - .row(blocked_index.block_offset) - }); - - let group_idx = match entry { - // Existing group_index for this group value - Some((_hash, group_idx)) => *group_idx, - // 1.2 Need to create new entry for the group - None => { - // Add new entry to aggr_state and save newly created index - if let Some(blk_size) = self.block_size { - if group_values.current().unwrap().num_rows() == blk_size { - // Use blk_size as offset cap, - // and old block's buffer size as buffer cap - let new_buf_cap = - rows_buffer_size(group_values.current().unwrap()); - let new_blk = - self.row_converter.empty_rows(blk_size, new_buf_cap); - group_values.push_block(new_blk); + let blk_id = group_values.num_blocks() - 1; + let cur_blk = group_values.current_mut().unwrap(); + let blk_offset = cur_blk.num_rows(); + cur_blk.push(group_rows.row(row)); + + let blocked_index = BlockedGroupIndex::new_from_parts( + blk_id, + blk_offset, + self.block_size.is_some(), + ); + let group_idx = blocked_index.as_packed_index(); + + // for hasher function, use precomputed hash value + self.map.insert_accounted( + (target_hash, group_idx), + |(hash, _group_index)| *hash, + &mut self.map_size, + ); + + group_idx } - } - - let blk_id = group_values.num_blocks() - 1; - let cur_blk = group_values.current_mut().unwrap(); - let blk_offset = cur_blk.num_rows(); - cur_blk.push(group_rows.row(row)); - - let blocked_index = BlockedGroupIndex::new_from_parts( - blk_id, - blk_offset, - self.block_size.is_some(), - ); - let group_idx = blocked_index.as_packed_index(); - - // for hasher function, use precomputed hash value - self.map.insert_accounted( - (target_hash, group_idx), - |(hash, _group_index)| *hash, - &mut self.map_size, - ); - - group_idx + }; + groups.push(group_idx); } }; - groups.push(group_idx); - } + + if self.block_size.is_some() { + get_or_create_groups(BlockedGroupIndex::new_blocked); + } else { + get_or_create_groups(BlockedGroupIndex::new_flat); + }; self.group_values = group_values; From 3316f8f92370521641c495ea12cedbf4b196e22d Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 00:42:09 +0800 Subject: [PATCH 070/107] continue to eliminate more dynamic dispatch. --- .../src/aggregates/group_values/row.rs | 46 ++++++++++--------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 9cdd1b1f4c86..09c469b73dfc 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -294,32 +294,34 @@ impl GroupValues for GroupValuesRows { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; + let mut shift_down_values = + |group_index_parse_fn: fn(usize) -> BlockedGroupIndex| unsafe { + for bucket in self.map.iter() { + // Decrement group index by n + let group_idx = bucket.as_ref().1; + let old_blk_idx = group_index_parse_fn(group_idx); + match old_blk_idx.block_id.checked_sub(1) { + // Group index was >= n, shift value down + Some(new_blk_id) => { + let new_group_idx = BlockedGroupIndex::new_from_parts( + new_blk_id, + old_blk_idx.block_offset, + true, + ); + bucket.as_mut().1 = new_group_idx.as_packed_index(); + } + // Group index was < n, so remove from table + None => self.map.erase(bucket), + } + } + }; + let group_index_parse_fn = if self.block_size.is_some() { - BlockedGroupIndex::new_blocked + shift_down_values(BlockedGroupIndex::new_blocked); } else { - BlockedGroupIndex::new_flat + shift_down_values(BlockedGroupIndex::new_flat); }; - unsafe { - for bucket in self.map.iter() { - // Decrement group index by n - let group_idx = bucket.as_ref().1; - let old_blk_idx = group_index_parse_fn(group_idx); - match old_blk_idx.block_id.checked_sub(1) { - // Group index was >= n, shift value down - Some(new_blk_id) => { - let new_group_idx = BlockedGroupIndex::new_from_parts( - new_blk_id, - old_blk_idx.block_offset, - true, - ); - bucket.as_mut().1 = new_group_idx.as_packed_index(); - } - // Group index was < n, so remove from table - None => self.map.erase(bucket), - } - } - } output } EmitTo::NextBlock(false) => { From 8a8e799589271436992a8bf7bee01acab9632369 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 00:43:51 +0800 Subject: [PATCH 071/107] test. --- datafusion/common/src/config.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ee5cc2695a54..b88c50cbbbfd 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -353,7 +353,7 @@ config_namespace! { /// next block rather than growing current block and copying the data. This /// approach can eliminate all unnecessary copies and get a good performance finally. /// We plan to make this the default in the future when tests are enough. - pub enable_aggregation_group_states_blocked_approach: bool, default = false + pub enable_aggregation_group_states_blocked_approach: bool, default = true } } From 921cad76b1c2df4e8a4799f3b216bb3feda53ced Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 01:06:43 +0800 Subject: [PATCH 072/107] fix clippy. --- datafusion/physical-plan/src/aggregates/group_values/row.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index 09c469b73dfc..bbf1856f3b78 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -316,7 +316,7 @@ impl GroupValues for GroupValuesRows { } }; - let group_index_parse_fn = if self.block_size.is_some() { + if self.block_size.is_some() { shift_down_values(BlockedGroupIndex::new_blocked); } else { shift_down_values(BlockedGroupIndex::new_flat); From 8c5afa9392ceb8bc3a8093d920f85f121992d64c Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 01:07:02 +0800 Subject: [PATCH 073/107] fix sql logic tests. --- datafusion/sqllogictest/test_files/information_schema.slt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index efd017a90bc4..2353756dee69 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -173,6 +173,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false +datafusion.execution.enable_aggregation_group_states_blocked_approach false datafusion.execution.enable_recursive_ctes true datafusion.execution.keep_partition_by_columns false datafusion.execution.listing_table_ignore_subdirectory true @@ -263,6 +264,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold f 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_group_states_blocked_approach false Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, and such group values and states will be managed using a single big block(can think a `Vec`), obviously as the block growing up, many copies will be triggered and finally get a bad performance. If setting this flag to `true`, the blocked approach will be used. We will allocate the `block size` capacity for block first, and when we found the block has been filled to `block size` limit, we will allocate next block rather than growing current block and copying the data. This approach can eliminate all unnecessary copies and get a good performance finally. We plan to make this the default in the future when tests are enough. datafusion.execution.enable_recursive_ctes true Should DataFusion support recursive CTEs datafusion.execution.keep_partition_by_columns false Should DataFusion keep the columns used for partition_by in the output RecordBatches 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`). From e4dd31a8ab329254d4a36ba19f25d619b0697f4a Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 01:07:20 +0800 Subject: [PATCH 074/107] add options to enable blocked apporach in benmarks. --- benchmarks/src/clickbench.rs | 6 ++++++ benchmarks/src/tpch/run.rs | 5 +++++ benchmarks/src/util/options.rs | 5 +++++ datafusion/common/src/config.rs | 2 +- 4 files changed, 17 insertions(+), 1 deletion(-) diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index a0f051d17623..fa6b7ffafeb7 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -123,6 +123,12 @@ impl RunOpt { .parquet .schema_force_string_view = self.common.string_view; + config + .options_mut() + .execution + .enable_aggregation_group_states_blocked_approach = + self.common.blocked_approach; + let ctx = SessionContext::new_with_config(config); self.register_hits(&ctx).await?; diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index ebc5ac0dbd5a..91ea0ed917e8 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -125,6 +125,11 @@ impl RunOpt { .execution .parquet .schema_force_string_view = self.common.string_view; + config + .options_mut() + .execution + .enable_aggregation_group_states_blocked_approach = + self.common.blocked_approach; let ctx = SessionContext::new_with_config(config); // register tables diff --git a/benchmarks/src/util/options.rs b/benchmarks/src/util/options.rs index 02591e293272..af7fc6ef87d2 100644 --- a/benchmarks/src/util/options.rs +++ b/benchmarks/src/util/options.rs @@ -42,6 +42,11 @@ pub struct CommonOpt { /// when reading ParquetFiles #[structopt(long)] pub string_view: bool, + + /// If true, will use blocked approach to manage group values and states + /// in accumulators in aggregation. + #[structopt(long)] + pub blocked_approach: bool, } impl CommonOpt { diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index b88c50cbbbfd..ee5cc2695a54 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -353,7 +353,7 @@ config_namespace! { /// next block rather than growing current block and copying the data. This /// approach can eliminate all unnecessary copies and get a good performance finally. /// We plan to make this the default in the future when tests are enough. - pub enable_aggregation_group_states_blocked_approach: bool, default = true + pub enable_aggregation_group_states_blocked_approach: bool, default = false } } From db431cb3ce5268aa24e0951ea6db3451cb092f73 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 02:39:22 +0800 Subject: [PATCH 075/107] fix fmt and clippy. --- .../src/aggregate/groups_accumulator.rs | 31 +++++++++++-------- .../groups_accumulator/accumulate.rs | 18 +++++++---- .../aggregate/groups_accumulator/prim_op.rs | 3 +- datafusion/functions-aggregate/src/average.rs | 16 +++++----- datafusion/functions-aggregate/src/count.rs | 16 +++++----- .../src/aggregates/group_values/row.rs | 12 +++---- 6 files changed, 54 insertions(+), 42 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index ee24379042b3..3c8ca3d53910 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -446,18 +446,14 @@ impl EmitToExt for EmitTo { /// - Low 32 bits represent `block_offset` #[derive(Debug, Clone, Copy)] pub struct BlockedGroupIndex { - pub block_id: usize, - pub block_offset: usize, + pub block_id: u32, + pub block_offset: u64, pub is_blocked: bool, } impl BlockedGroupIndex { #[inline] - pub fn new_from_parts( - block_id: usize, - block_offset: usize, - is_blocked: bool, - ) -> Self { + pub fn new_from_parts(block_id: u32, block_offset: u64, is_blocked: bool) -> Self { Self { block_id, block_offset, @@ -469,16 +465,15 @@ impl BlockedGroupIndex { pub fn new_flat(raw_index: usize) -> Self { Self { block_id: 0, - block_offset: raw_index, + block_offset: raw_index as u64, is_blocked: false, } } #[inline] pub fn new_blocked(raw_index: usize) -> Self { - let block_id = - ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; - let block_offset = ((raw_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK) as usize; + let block_id = ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as u32; + let block_offset = (raw_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK; Self { block_id, @@ -487,11 +482,21 @@ impl BlockedGroupIndex { } } + #[inline] + pub fn block_id(&self) -> usize { + self.block_id as usize + } + + #[inline] + pub fn block_offset(&self) -> usize { + self.block_offset as usize + } + pub fn as_packed_index(&self) -> usize { if self.is_blocked { - (((self.block_id as u64) << 32) | (self.block_offset as u64)) as usize + (((self.block_id as u64) << 32) | self.block_offset) as usize } else { - self.block_offset + self.block_offset as usize } } } 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 4af736084361..269322b1e37d 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -427,7 +427,8 @@ impl BlockedNullState { BlockedGroupIndex::new_blocked, value_fn, |index: &BlockedGroupIndex| { - seen_values_blocks[index.block_id].set_bit(index.block_offset, true); + seen_values_blocks[index.block_id()] + .set_bit(index.block_offset(), true); }, ) } else { @@ -438,7 +439,8 @@ impl BlockedNullState { BlockedGroupIndex::new_flat, value_fn, |index: &BlockedGroupIndex| { - seen_values_blocks[index.block_id].set_bit(index.block_offset, true); + seen_values_blocks[index.block_id()] + .set_bit(index.block_offset(), true); }, ); } @@ -939,8 +941,12 @@ mod test { .map(|idx| { let block_id = *idx / self.block_size; let block_offset = *idx % self.block_size; - BlockedGroupIndex::new_from_parts(block_id, block_offset, true) - .as_packed_index() + BlockedGroupIndex::new_from_parts( + block_id as u32, + block_offset as u64, + true, + ) + .as_packed_index() }) .collect::>(); @@ -950,8 +956,8 @@ mod test { opt_filter, total_num_groups, |blocked_index, value| { - let flat_index = blocked_index.block_id * self.block_size - + blocked_index.block_offset; + let flat_index = blocked_index.block_id() * self.block_size + + blocked_index.block_offset(); accumulated_values.push((flat_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 5f1b9ae2c396..0788ab619556 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 @@ -119,7 +119,8 @@ where opt_filter, total_num_groups, |index, new_value| { - let value = &mut self.values_blocks[index.block_id][index.block_offset]; + let value = + &mut self.values_blocks[index.block_id()][index.block_offset()]; (self.prim_fn)(value, new_value); }, ); diff --git a/datafusion/functions-aggregate/src/average.rs b/datafusion/functions-aggregate/src/average.rs index 5c6d447e4b15..b6ba14dfcb32 100644 --- a/datafusion/functions-aggregate/src/average.rs +++ b/datafusion/functions-aggregate/src/average.rs @@ -475,10 +475,10 @@ where opt_filter, total_num_groups, |blocked_index, new_value| { - let sum = - &mut self.sums[blocked_index.block_id][blocked_index.block_offset]; - let count = - &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + let sum = &mut self.sums[blocked_index.block_id()] + [blocked_index.block_offset()]; + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; *sum = sum.add_wrapping(new_value); *count += 1; @@ -579,8 +579,8 @@ where opt_filter, total_num_groups, |blocked_index, partial_count| { - let count = - &mut self.counts[blocked_index.block_id][blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; *count += partial_count; }, ); @@ -591,8 +591,8 @@ where opt_filter, total_num_groups, |blocked_index, new_value: ::Native| { - let sum = - &mut self.sums[blocked_index.block_id][blocked_index.block_offset]; + let sum = &mut self.sums[blocked_index.block_id()] + [blocked_index.block_offset()]; *sum = sum.add_wrapping(new_value); }, ); diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index 6be06acf8a16..d9cd3988129c 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -406,8 +406,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter, |group_index| { let blocked_index = BlockedGroupIndex::new_blocked(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; *count += 1; }, ); @@ -418,8 +418,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter, |group_index| { let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; *count += 1; }, ); @@ -457,8 +457,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter, |group_index, partial_count| { let blocked_index = BlockedGroupIndex::new_blocked(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; *count += partial_count; }, ); @@ -469,8 +469,8 @@ impl GroupsAccumulator for CountGroupsAccumulator { opt_filter, |group_index, partial_count| { let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = &mut self.counts[blocked_index.block_id] - [blocked_index.block_offset]; + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; *count += partial_count; }, ); diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index bbf1856f3b78..ea55ac5d450d 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -153,8 +153,8 @@ impl GroupValues for GroupValuesRows { // existing_idx (aka group_values @ row) let blocked_index = group_index_parse_fn(*group_idx); group_rows.row(row) - == group_values[blocked_index.block_id] - .row(blocked_index.block_offset) + == group_values[blocked_index.block_id()] + .row(blocked_index.block_offset()) }); let group_idx = match entry { @@ -183,8 +183,8 @@ impl GroupValues for GroupValuesRows { cur_blk.push(group_rows.row(row)); let blocked_index = BlockedGroupIndex::new_from_parts( - blk_id, - blk_offset, + blk_id as u32, + blk_offset as u64, self.block_size.is_some(), ); let group_idx = blocked_index.as_packed_index(); @@ -300,11 +300,11 @@ impl GroupValues for GroupValuesRows { // Decrement group index by n let group_idx = bucket.as_ref().1; let old_blk_idx = group_index_parse_fn(group_idx); - match old_blk_idx.block_id.checked_sub(1) { + match old_blk_idx.block_id().checked_sub(1) { // Group index was >= n, shift value down Some(new_blk_id) => { let new_group_idx = BlockedGroupIndex::new_from_parts( - new_blk_id, + new_blk_id as u32, old_blk_idx.block_offset, true, ); From f2e316a85d659ec7927e47790f267e363146ebf2 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 02:42:49 +0800 Subject: [PATCH 076/107] fix tpch opts. --- benchmarks/src/tpch/run.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index 91ea0ed917e8..ba1af56d05d3 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -350,6 +350,7 @@ mod tests { batch_size: 8192, debug: false, string_view: false, + blocked_approach: false, }; let opt = RunOpt { query: Some(query), @@ -384,6 +385,7 @@ mod tests { batch_size: 8192, debug: false, string_view: false, + blocked_approach: false, }; let opt = RunOpt { query: Some(query), From 8b8da5edf82447526062bc84e4dc5cf2f2367af1 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 02:47:36 +0800 Subject: [PATCH 077/107] fix comments of `BlockedGroupIndex`. --- .../src/aggregate/groups_accumulator.rs | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 3c8ca3d53910..f51f8f0fe4e1 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -441,9 +441,14 @@ impl EmitToExt for EmitTo { /// Blocked style group index used in blocked mode group values and accumulators /// -/// Parts in index: +/// In blocked mode(is_blocked=true): /// - High 32 bits represent `block_id` /// - Low 32 bits represent `block_offset` +/// +/// In flat mode(is_blocked=false) +/// - `block_id` is always 0 +/// - total 64 bits used to represent the `block offset` +/// #[derive(Debug, Clone, Copy)] pub struct BlockedGroupIndex { pub block_id: u32, From 21e5fdfe4daa92f8680678c4fd6db63cf6e6ca78 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 02:50:32 +0800 Subject: [PATCH 078/107] update config. --- docs/source/user-guide/configs.md | 176 +++++++++++++++--------------- 1 file changed, 89 insertions(+), 87 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 4255307781b6..896468e32fea 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,90 +35,92 @@ 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.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.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 | -| 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 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| 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.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.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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| 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, and Ansi. | -| 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. | +| 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.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.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 | +| 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 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | +| 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.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.enable_aggregation_group_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, and such group values and states will be managed using a single big block(can think a `Vec`), obviously as the block growing up, many copies will be triggered and finally get a bad performance. If setting this flag to `true`, the blocked approach will be used. We will allocate the `block size` capacity for block first, and when we found the block has been filled to `block size` limit, we will allocate next block rather than growing current block and copying the data. This approach can eliminate all unnecessary copies and get a good performance finally. We plan to make this the default in the future when tests are enough. | +| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| 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, and Ansi. | +| 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. | + From 8d5cb7fc2a6f2279f1fcc42ba3c06527e7acaeea Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 03:20:56 +0800 Subject: [PATCH 079/107] fix docs. --- .../src/aggregate/groups_accumulator.rs | 2 +- docs/source/user-guide/configs.md | 177 +++++++++--------- 2 files changed, 89 insertions(+), 90 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index f51f8f0fe4e1..af838c7b86ba 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -447,7 +447,7 @@ impl EmitToExt for EmitTo { /// /// In flat mode(is_blocked=false) /// - `block_id` is always 0 -/// - total 64 bits used to represent the `block offset` +/// - Total 64 bits used to represent the `block offset` /// #[derive(Debug, Clone, Copy)] pub struct BlockedGroupIndex { diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 896468e32fea..17a0aac3bc85 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,92 +35,91 @@ 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.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.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 | -| 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 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| 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.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.enable_aggregation_group_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, and such group values and states will be managed using a single big block(can think a `Vec`), obviously as the block growing up, many copies will be triggered and finally get a bad performance. If setting this flag to `true`, the blocked approach will be used. We will allocate the `block size` capacity for block first, and when we found the block has been filled to `block size` limit, we will allocate next block rather than growing current block and copying the data. This approach can eliminate all unnecessary copies and get a good performance finally. We plan to make this the default in the future when tests are enough. | -| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| 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, and Ansi. | -| 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. | - +| 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.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.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 | +| 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 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | +| 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.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.enable_aggregation_group_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, and such group values and states will be managed using a single big block(can think a `Vec`), obviously as the block growing up, many copies will be triggered and finally get a bad performance. If setting this flag to `true`, the blocked approach will be used. We will allocate the `block size` capacity for block first, and when we found the block has been filled to `block size` limit, we will allocate next block rather than growing current block and copying the data. This approach can eliminate all unnecessary copies and get a good performance finally. We plan to make this the default in the future when tests are enough. | +| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| 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, and Ansi. | +| 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. | From b33c6f908dc56dcbcbb37b193b55d64ad172fe69 Mon Sep 17 00:00:00 2001 From: kamille Date: Mon, 26 Aug 2024 03:41:51 +0800 Subject: [PATCH 080/107] use the right way to check if spilling enalbed, and support `emit_early_if_necessary` again. --- .../src/aggregate/groups_accumulator.rs | 2 +- .../physical-plan/src/aggregates/row_hash.rs | 24 +++++++++++-------- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index af838c7b86ba..f51f8f0fe4e1 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -447,7 +447,7 @@ impl EmitToExt for EmitTo { /// /// In flat mode(is_blocked=false) /// - `block_id` is always 0 -/// - Total 64 bits used to represent the `block offset` +/// - total 64 bits used to represent the `block offset` /// #[derive(Debug, Clone, Copy)] pub struct BlockedGroupIndex { diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index bcff005a383b..1524b0b3a719 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -42,9 +42,9 @@ use arrow_schema::SortOptions; use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_execution::disk_manager::RefCountedTempFile; use datafusion_execution::memory_pool::proxy::VecAllocExt; -use datafusion_execution::memory_pool::{MemoryConsumer, MemoryPool, MemoryReservation}; +use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; -use datafusion_execution::TaskContext; +use datafusion_execution::{DiskManager, TaskContext}; use datafusion_expr::{EmitTo, GroupsAccumulator}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ @@ -620,7 +620,7 @@ fn maybe_enable_blocked_group_states( .enable_aggregation_group_states_blocked_approach || !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() - || enable_spilling(context.memory_pool().as_ref()) + || enable_spilling(context.runtime_env().disk_manager.as_ref()) { return Ok(false); } @@ -641,9 +641,9 @@ fn maybe_enable_blocked_group_states( } } -// TODO: we should add a function(like `name`) to distinguish different memory pools. -fn enable_spilling(memory_pool: &dyn MemoryPool) -> bool { - !format!("{memory_pool:?}").contains("UnboundedMemoryPool") +#[inline] +fn enable_spilling(disk_manager: &DiskManager) -> bool { + disk_manager.tmp_files_enabled() } /// Create an accumulator for `agg_expr` -- a [`GroupsAccumulator`] if @@ -1058,11 +1058,15 @@ impl GroupedHashAggregateStream { && matches!(self.mode, AggregateMode::Partial) && self.update_memory_reservation().is_err() { - let n = self.group_values.len() / self.batch_size * self.batch_size; - let batch = self.emit(EmitTo::First(n), false)?; - self.exec_state = ExecutionState::ProducingOutput(batch); + if !self.enable_blocked_group_states { + let n = self.group_values.len() / self.batch_size * self.batch_size; + let batch = self.emit(EmitTo::First(n), false)?; + self.exec_state = ExecutionState::ProducingOutput(batch); + } else { + let blocks = self.group_values.len() / self.batch_size; + self.exec_state = ExecutionState::ProducingBlocks(Some(blocks)); + } } - Ok(()) } From fd54e1ca54f9961e84ed07d2917826be45a6857f Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 00:17:54 +0800 Subject: [PATCH 081/107] unify ensure_enough_room_for_xxx and add tests. --- .../src/aggregate/groups_accumulator.rs | 118 ++++++------------ .../groups_accumulator/accumulate.rs | 76 ++++------- .../aggregate/groups_accumulator/prim_op.rs | 1 + 3 files changed, 68 insertions(+), 127 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index f51f8f0fe4e1..a8ec1853503b 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -29,6 +29,7 @@ use std::{ fmt::{self, Debug}, iter, ops::{Index, IndexMut}, + usize, }; use arrow::{ @@ -43,8 +44,7 @@ use datafusion_common::{ use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; -pub const BLOCKED_INDEX_HIGH_32_BITS_MASK: u64 = 0xffffffff00000000; -pub const BLOCKED_INDEX_LOW_32_BITS_MASK: u64 = 0x00000000ffffffff; +pub const MAX_PREALLOC_BLOCK_SIZE: usize = 8192; /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] /// @@ -477,8 +477,8 @@ impl BlockedGroupIndex { #[inline] pub fn new_blocked(raw_index: usize) -> Self { - let block_id = ((raw_index as u64 >> 32) & BLOCKED_INDEX_LOW_32_BITS_MASK) as u32; - let block_offset = (raw_index as u64) & BLOCKED_INDEX_LOW_32_BITS_MASK; + let block_id = ((raw_index as u64 >> 32) & 0x00000000ffffffff) as u32; + let block_offset = (raw_index as u64) & 0x00000000ffffffff; Self { block_id, @@ -719,52 +719,6 @@ pub(crate) fn slice_and_maybe_filter( } } -pub fn ensure_enough_room_for_values( - values: &mut VecBlocks, - total_num_groups: usize, - block_size: Option, - default_value: T, -) { - match block_size { - Some(blk_size) => ensure_enough_room_for_blocked_values( - values, - total_num_groups, - blk_size, - default_value, - ), - None => { - ensure_enough_room_for_flat_values(values, total_num_groups, default_value) - } - } -} - -/// Expend flat values to a big enough size for holding `total_num_groups` groups. -/// -/// For example, -/// -/// before expanding: -/// values: [x, x, x, x, x, x] -/// total_num_groups: 8 -/// -/// After expanding: -/// values: [x, x, x, x, x, x, default, default] -/// -pub fn ensure_enough_room_for_flat_values( - values: &mut VecBlocks, - total_num_groups: usize, - default_value: T, -) { - // It flat mode, we just a single builder, and grow it constantly. - if values.num_blocks() == 0 { - values.push_block(Vec::new()); - } - - values - .current_mut() - .unwrap() - .resize(total_num_groups, default_value); -} - /// Expend blocked values to a big enough size for holding `total_num_groups` groups. /// /// For example, @@ -776,12 +730,13 @@ pub fn ensure_enough_room_for_flat_values( /// After expanding: /// values: [x, x, x], [x, x, x], [default, default, default] /// -pub fn ensure_enough_room_for_blocked_values( +pub fn ensure_enough_room_for_values( values: &mut VecBlocks, total_num_groups: usize, - block_size: usize, + block_size: Option, default_value: T, ) { + let block_size = block_size.unwrap_or(usize::MAX); // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { @@ -801,10 +756,13 @@ pub fn ensure_enough_room_for_blocked_values( // Ensure blks are enough. let exist_blks = values.num_blocks(); - let new_blks = (total_num_groups + block_size - 1) / block_size - exist_blks; + let new_blks = + total_num_groups.saturating_add(block_size - 1) / block_size - exist_blks; + if new_blks > 0 { for _ in 0..new_blks { - values.push_block(Vec::with_capacity(block_size)); + values + .push_block(Vec::with_capacity(block_size.min(MAX_PREALLOC_BLOCK_SIZE))); } } @@ -847,32 +805,39 @@ mod tests { use super::*; #[test] - fn test_ensure_room_for_blocked_values() { + fn test_ensure_room_for_values() { let mut blocks = VecBlocks::new(); let block_size = 4; // 0 total_num_groups, should be no blocks - ensure_enough_room_for_blocked_values(&mut blocks, 0, block_size, 0); + ensure_enough_room_for_values(&mut blocks, 0, Some(block_size), 0); assert_eq!(blocks.num_blocks(), 0); assert_eq!(blocks.len(), 0); // 0 -> 3 total_num_groups, blocks should look like: // [d, d, d, empty] - ensure_enough_room_for_blocked_values(&mut blocks, 3, block_size, 0); + ensure_enough_room_for_values(&mut blocks, 3, Some(block_size), 0); assert_eq!(blocks.num_blocks(), 1); assert_eq!(blocks.len(), 3); // 3 -> 8 total_num_groups, blocks should look like: // [d, d, d, d], [d, d, d, d] - ensure_enough_room_for_blocked_values(&mut blocks, 8, block_size, 0); + ensure_enough_room_for_values(&mut blocks, 8, Some(block_size), 0); assert_eq!(blocks.num_blocks(), 2); assert_eq!(blocks.len(), 8); // 8 -> 13 total_num_groups, blocks should look like: // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] - ensure_enough_room_for_blocked_values(&mut blocks, 13, block_size, 0); + ensure_enough_room_for_values(&mut blocks, 13, Some(block_size), 0); assert_eq!(blocks.num_blocks(), 4); assert_eq!(blocks.len(), 13); + + // Block size none, it means we will always use one single block + // [] -> [d, d, d,...,d] + blocks.clear(); + ensure_enough_room_for_values(&mut blocks, 42, None, 0); + assert_eq!(blocks.num_blocks(), 1); + assert_eq!(blocks.len(), 42); } #[test] @@ -959,27 +924,24 @@ mod tests { assert!(origin.is_empty()); } - // #[test] - // fn test_take_need_from_blocks() { - // let block1 = vec![1, 2, 3, 4]; - // let block2 = vec![5, 6, 7, 8]; + #[test] + fn test_take_need_from_blocks() { + let block1 = vec![1, 2, 3, 4]; + let block2 = vec![5, 6, 7, 8]; - // let mut values = VecBlocks::new(); - // values.push_block(block1.clone()); - // values.push_block(block2.clone()); + let mut values = VecBlocks::new(); + values.push_block(block1.clone()); + values.push_block(block2.clone()); - // // Test emit block - // let emit = EmitTo::NextBlock(false); - // let actual = - // emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); - // assert_eq!(actual, block1); + // Test emit block + let emit = EmitTo::NextBlock(false); + let actual = emit.take_needed_from_blocks(&mut values); + assert_eq!(actual, block1); - // let actual = - // emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); - // assert_eq!(actual, block2); + let actual = emit.take_needed_from_blocks(&mut values); + assert_eq!(actual, block2); - // let actual = - // emit.take_needed_from_blocks(&mut values, GroupStatesMode::Blocked(4)); - // assert!(actual.is_empty()); - // } + let actual = emit.take_needed_from_blocks(&mut values); + assert!(actual.is_empty()); + } } 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 269322b1e37d..455fd0a1ba34 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,13 +19,18 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator +use std::usize; + 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::{BlockedGroupIndex, Blocks}; +use crate::aggregate::groups_accumulator::{ + BlockedGroupIndex, Blocks, MAX_PREALLOC_BLOCK_SIZE, +}; + /// 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. /// @@ -703,53 +708,14 @@ fn initialize_builder( builder } +/// Similar as the [initialize_builder] but designed for the blocked version accumulator fn ensure_enough_room_for_nulls( builder_blocks: &mut Blocks, total_num_groups: usize, block_size: Option, default_value: bool, ) { - match block_size { - Some(blk_size) => ensure_enough_room_for_blocked_nulls( - builder_blocks, - total_num_groups, - blk_size, - default_value, - ), - None => ensure_enough_room_for_flat_nulls( - builder_blocks, - total_num_groups, - default_value, - ), - } -} - -/// Similar as the [initialize_builder] -fn ensure_enough_room_for_flat_nulls( - builder_blocks: &mut Blocks, - total_num_groups: usize, - - default_value: bool, -) { - // It flat mode, we just a single builder, and grow it constantly. - if builder_blocks.num_blocks() == 0 { - builder_blocks.push_block(BooleanBufferBuilder::new(0)); - } - - let builder = builder_blocks.current_mut().unwrap(); - if builder.len() < total_num_groups { - let new_groups = total_num_groups - builder.len(); - builder.append_n(new_groups, default_value); - } -} - -/// Similar as the [initialize_builder] but designed for the blocked version accumulator -fn ensure_enough_room_for_blocked_nulls( - builder_blocks: &mut Blocks, - total_num_groups: usize, - block_size: usize, - default_value: bool, -) { + let block_size = block_size.unwrap_or(usize::MAX); // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { @@ -769,10 +735,13 @@ fn ensure_enough_room_for_blocked_nulls( // Ensure blks are enough let exist_blks = builder_blocks.num_blocks(); - let new_blks = (total_num_groups + block_size - 1) / block_size - exist_blks; + let new_blks = + (total_num_groups.saturating_add(block_size - 1) / block_size) - exist_blks; if new_blks > 0 { for _ in 0..new_blks { - builder_blocks.push_block(BooleanBufferBuilder::new(block_size)); + builder_blocks.push_block(BooleanBufferBuilder::new( + block_size.min(MAX_PREALLOC_BLOCK_SIZE), + )); } } @@ -1492,35 +1461,44 @@ mod test { } #[test] - fn test_ensure_room_for_blocked_nulls() { + fn test_ensure_room_for_nulls() { let mut blocks: Blocks = Blocks::new(); let block_size = 4; + // Block size < usize::MAX // 0 total_num_groups, should be no blocks - ensure_enough_room_for_blocked_nulls(&mut blocks, 0, block_size, false); + ensure_enough_room_for_nulls(&mut blocks, 0, Some(block_size), false); assert_eq!(blocks.num_blocks(), 0); let total_len = blocks.iter().map(|blk| blk.len()).sum::(); assert_eq!(total_len, 0); // 0 -> 3 total_num_groups, blocks should look like: // [d, d, d, empty] - ensure_enough_room_for_blocked_nulls(&mut blocks, 3, block_size, false); + ensure_enough_room_for_nulls(&mut blocks, 3, Some(block_size), false); assert_eq!(blocks.num_blocks(), 1); let total_len = blocks.iter().map(|blk| blk.len()).sum::(); assert_eq!(total_len, 3); // 3 -> 8 total_num_groups, blocks should look like: // [d, d, d, d], [d, d, d, d] - ensure_enough_room_for_blocked_nulls(&mut blocks, 8, block_size, false); + ensure_enough_room_for_nulls(&mut blocks, 8, Some(block_size), false); assert_eq!(blocks.num_blocks(), 2); let total_len = blocks.iter().map(|blk| blk.len()).sum::(); assert_eq!(total_len, 8); // 8 -> 13 total_num_groups, blocks should look like: // [d, d, d, d], [d, d, d, d], [d, d, d, d], [d, empty, empty, empty] - ensure_enough_room_for_blocked_nulls(&mut blocks, 13, block_size, false); + ensure_enough_room_for_nulls(&mut blocks, 13, Some(block_size), false); assert_eq!(blocks.num_blocks(), 4); let total_len = blocks.iter().map(|blk| blk.len()).sum::(); assert_eq!(total_len, 13); + + // Block size none, it means we will always use one single block + // [] -> [d, d, d,...,d] + blocks.clear(); + ensure_enough_room_for_nulls(&mut blocks, 42, None, false); + assert_eq!(blocks.num_blocks(), 1); + let total_len = blocks.iter().map(|blk| blk.len()).sum::(); + assert_eq!(total_len, 42); } } 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 0788ab619556..2ac2d9871027 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 @@ -16,6 +16,7 @@ // under the License. use std::sync::Arc; +use std::usize; use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; use arrow::buffer::NullBuffer; From addcc1391da1d541858e77032fcde74fe6e49668 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 00:54:27 +0800 Subject: [PATCH 082/107] fix ensure_enough_room_for_xxx. --- .../src/aggregate/groups_accumulator.rs | 21 ++++++++++--------- .../groups_accumulator/accumulate.rs | 20 ++++++++++-------- 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index a8ec1853503b..a80b4bbf5f72 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -736,13 +736,13 @@ pub fn ensure_enough_room_for_values( block_size: Option, default_value: T, ) { - let block_size = block_size.unwrap_or(usize::MAX); + let calc_block_size = block_size.unwrap_or(usize::MAX); // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. let (mut cur_blk_idx, exist_slots) = if values.num_blocks() > 0 { let cur_blk_idx = values.num_blocks() - 1; let exist_slots = - (values.num_blocks() - 1) * block_size + values.current().unwrap().len(); + (values.num_blocks() - 1) * calc_block_size + values.current().unwrap().len(); (cur_blk_idx, exist_slots) } else { @@ -756,13 +756,13 @@ pub fn ensure_enough_room_for_values( // Ensure blks are enough. let exist_blks = values.num_blocks(); - let new_blks = - total_num_groups.saturating_add(block_size - 1) / block_size - exist_blks; + let new_blks = total_num_groups.saturating_add(calc_block_size - 1) / calc_block_size + - exist_blks; if new_blks > 0 { + let prealloc_size = block_size.unwrap_or(0).min(MAX_PREALLOC_BLOCK_SIZE); for _ in 0..new_blks { - values - .push_block(Vec::with_capacity(block_size.min(MAX_PREALLOC_BLOCK_SIZE))); + values.push_block(Vec::with_capacity(calc_block_size.min(prealloc_size))); } } @@ -770,7 +770,7 @@ pub fn ensure_enough_room_for_values( let mut new_slots = total_num_groups - exist_slots; // Expand current blk. - let cur_blk_rest_slots = block_size - values[cur_blk_idx].len(); + let cur_blk_rest_slots = calc_block_size - values[cur_blk_idx].len(); if cur_blk_rest_slots >= new_slots { // We just need to expand current blocks. values[cur_blk_idx].extend(iter::repeat(default_value.clone()).take(new_slots)); @@ -784,14 +784,15 @@ pub fn ensure_enough_room_for_values( cur_blk_idx += 1; // Expand whole blks - let expand_blks = new_slots / block_size; + let expand_blks = new_slots / calc_block_size; for _ in 0..expand_blks { - values[cur_blk_idx].extend(iter::repeat(default_value.clone()).take(block_size)); + values[cur_blk_idx] + .extend(iter::repeat(default_value.clone()).take(calc_block_size)); cur_blk_idx += 1; } // Expand the last blk if needed - let last_expand_slots = new_slots % block_size; + let last_expand_slots = new_slots % calc_block_size; if last_expand_slots > 0 { values .current_mut() 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 455fd0a1ba34..65e07a762441 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -715,12 +715,12 @@ fn ensure_enough_room_for_nulls( block_size: Option, default_value: bool, ) { - let block_size = block_size.unwrap_or(usize::MAX); + let calc_block_size = block_size.unwrap_or(usize::MAX); // In blocked mode, we ensure the blks are enough first, // and then ensure slots in blks are enough. let (mut cur_blk_idx, exist_slots) = if builder_blocks.num_blocks() > 0 { let cur_blk_idx = builder_blocks.num_blocks() - 1; - let exist_slots = (builder_blocks.num_blocks() - 1) * block_size + let exist_slots = (builder_blocks.num_blocks() - 1) * calc_block_size + builder_blocks.current().unwrap().len(); (cur_blk_idx, exist_slots) @@ -735,12 +735,14 @@ fn ensure_enough_room_for_nulls( // Ensure blks are enough let exist_blks = builder_blocks.num_blocks(); - let new_blks = - (total_num_groups.saturating_add(block_size - 1) / block_size) - exist_blks; + let new_blks = (total_num_groups.saturating_add(calc_block_size - 1) + / calc_block_size) + - exist_blks; if new_blks > 0 { + let prealloc_size = block_size.unwrap_or(0).min(MAX_PREALLOC_BLOCK_SIZE); for _ in 0..new_blks { builder_blocks.push_block(BooleanBufferBuilder::new( - block_size.min(MAX_PREALLOC_BLOCK_SIZE), + calc_block_size.min(prealloc_size), )); } } @@ -749,7 +751,7 @@ fn ensure_enough_room_for_nulls( let mut new_slots = total_num_groups - exist_slots; // Expand current blk. - let cur_blk_rest_slots = block_size - builder_blocks[cur_blk_idx].len(); + let cur_blk_rest_slots = calc_block_size - builder_blocks[cur_blk_idx].len(); if cur_blk_rest_slots >= new_slots { builder_blocks[cur_blk_idx].append_n(new_slots, default_value); return; @@ -761,14 +763,14 @@ fn ensure_enough_room_for_nulls( cur_blk_idx += 1; // Expand blks - let expand_blks = new_slots / block_size; + let expand_blks = new_slots / calc_block_size; for _ in 0..expand_blks { - builder_blocks[cur_blk_idx].append_n(block_size, default_value); + builder_blocks[cur_blk_idx].append_n(calc_block_size, default_value); cur_blk_idx += 1; } // Expand the last blk. - let last_expand_slots = new_slots % block_size; + let last_expand_slots = new_slots % calc_block_size; if last_expand_slots > 0 { builder_blocks .current_mut() From 5a2292debcc1cee199cf7d13d9ca413e32743aae Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 02:11:17 +0800 Subject: [PATCH 083/107] add physical level test for blocked approach. --- .../physical-plan/src/aggregates/mod.rs | 126 ++++++++++++++++++ 1 file changed, 126 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 5aa255e7c341..911ea756a97f 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1207,6 +1207,7 @@ mod tests { ScalarValue, }; use datafusion_execution::config::SessionConfig; + use datafusion_execution::disk_manager::DiskManagerConfig; use datafusion_execution::memory_pool::FairSpillPool; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_functions_aggregate::array_agg::array_agg_udaf; @@ -2488,4 +2489,129 @@ mod tests { Ok(()) } + + #[tokio::test] + async fn test_blocked_approach() -> Result<()> { + // Define data + let schema = Arc::new(Schema::new(vec![ + Field::new("a1", DataType::UInt32, false), + Field::new("a2", DataType::UInt32, false), + Field::new("b", DataType::Float64, false), + ])); + + let input_data = vec![ + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(UInt32Array::from(vec![2, 3, 3, 4])), + Arc::new(UInt32Array::from(vec![4, 4, 3, 2])), + Arc::new(Float64Array::from(vec![1.0, 2.0, 3.0, 4.0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(UInt32Array::from(vec![2, 3, 3, 4])), + Arc::new(UInt32Array::from(vec![4, 4, 3, 2])), + Arc::new(Float64Array::from(vec![4.0, 3.0, 2.0, 1.0])), + ], + ) + .unwrap(), + RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(UInt32Array::from(vec![2, 3, 3, 4])), + Arc::new(UInt32Array::from(vec![2, 3, 3, 4])), + Arc::new(Float64Array::from(vec![5.0, 4.0, 3.0, 2.0])), + ], + ) + .unwrap(), + ]; + + // Define plan + let group_by = PhysicalGroupBy::new_single(vec![ + (col("a1", &schema)?, "a1".to_string()), + (col("a2", &schema)?, "a2".to_string()), + ]); + + let aggr_expr = + vec![ + AggregateExprBuilder::new(avg_udaf(), vec![col("b", &schema)?]) + .schema(Arc::clone(&schema)) + .alias(String::from("AVG(b)")) + .build()?, + ]; + + let input = Arc::new(MemoryExec::try_new( + &[input_data], + Arc::clone(&schema), + None, + )?); + + let partial_aggr_exec = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + group_by, + aggr_expr.clone(), + vec![None], + Arc::clone(&input) as Arc, + Arc::clone(&schema), + )?); + + let merge = Arc::new(CoalescePartitionsExec::new(partial_aggr_exec)); + + let final_group_by = PhysicalGroupBy::new_single(vec![ + (col("a1", &schema)?, "a1".to_string()), + (col("a2", &schema)?, "a2".to_string()), + ]); + + let merged_aggregate = Arc::new(AggregateExec::try_new( + AggregateMode::Final, + final_group_by, + aggr_expr, + vec![None], + merge, + schema, + )?); + + // Define task context + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_group_states_blocked_approach", + ScalarValue::Boolean(Some(true)), + ); + session_config = session_config.set( + "datafusion.execution.batch_size", + ScalarValue::UInt64(Some(1)), + ); + + let runtime = Arc::new( + RuntimeEnv::new( + RuntimeConfig::default().with_disk_manager(DiskManagerConfig::Disabled), + ) + .unwrap(), + ); + let ctx = TaskContext::default() + .with_session_config(session_config) + .with_runtime(runtime); + + // Run and check + let output = collect(merged_aggregate.execute(0, Arc::new(ctx))?).await?; + + let expected = [ + "+----+----+--------+", + "| a1 | a2 | AVG(b) |", + "+----+----+--------+", + "| 2 | 4 | 2.5 |", + "| 3 | 4 | 2.5 |", + "| 3 | 3 | 3.0 |", + "| 4 | 2 | 2.5 |", + "| 2 | 2 | 5.0 |", + "| 4 | 4 | 2.0 |", + "+----+----+--------+", + ]; + assert_batches_eq!(expected, &output); + + Ok(()) + } } From 207a777ff3bb3bf2305cab176150d30bd8b3d210 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 03:08:26 +0800 Subject: [PATCH 084/107] extract `run_aggregate_test_internal` for resuing later. --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 105 +++++++++++++----- 1 file changed, 79 insertions(+), 26 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 138e5bda7f39..c9932f442c21 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -33,6 +33,7 @@ use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::{DataFrame, SessionConfig, SessionContext}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; +use datafusion_execution::TaskContext; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; @@ -65,7 +66,7 @@ async fn streaming_aggregate_test() { for i in 0..n { let test_idx = i % test_cases.len(); let group_by_columns = test_cases[test_idx].clone(); - join_set.spawn(run_aggregate_test( + join_set.spawn(run_streaming_aggregate_test( make_staggered_batches::(1000, distinct, i as u64), group_by_columns, )); @@ -77,13 +78,13 @@ async fn streaming_aggregate_test() { } } -/// Perform batch and streaming aggregation with same input -/// and verify outputs of `AggregateExec` with pipeline breaking stream `GroupedHashAggregateStream` -/// and non-pipeline breaking stream `BoundedAggregateStream` produces same result. -async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str>) { - let schema = input1[0].schema(); - let session_config = SessionConfig::new().with_batch_size(50); - let ctx = SessionContext::new_with_config(session_config); +async fn run_streaming_aggregate_test( + test_data: Vec, + group_by_columns: Vec<&str>, +) { + let schema = test_data[0].schema(); + + // Define test data source exec let mut sort_keys = vec![]; for ordering_col in ["a", "b", "c"] { sort_keys.push(PhysicalSortExpr { @@ -92,17 +93,66 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str }) } - let concat_input_record = concat_batches(&schema, &input1).unwrap(); + let concat_input_record = concat_batches(&schema, &test_data).unwrap(); let usual_source = Arc::new( MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None).unwrap(), ); let running_source = Arc::new( - MemoryExec::try_new(&[input1.clone()], schema.clone(), None) + MemoryExec::try_new(&[test_data.clone()], schema.clone(), None) .unwrap() .with_sort_information(vec![sort_keys]), ); + // Define test task ctx + let session_config = SessionConfig::new().with_batch_size(50); + let ctx = SessionContext::new_with_config(session_config); + + // Run and check + let usual_aggr_ctx = AggrTestContext { + data_source_exec: usual_source, + task_ctx: ctx.task_ctx(), + }; + + let running_aggr_ctx = AggrTestContext { + data_source_exec: running_source, + task_ctx: ctx.task_ctx(), + }; + + run_aggregate_test_internal( + test_data, + usual_aggr_ctx, + running_aggr_ctx, + |collected_usual, collected_running| { + assert!(collected_running.len() > 2); + // Running should produce more chunk than the usual AggregateExec. + // Otherwise it means that we cannot generate result in running mode. + assert!(collected_running.len() > collected_usual.len()); + }, + group_by_columns, + ) + .await; +} + +struct AggrTestContext { + data_source_exec: Arc, + task_ctx: Arc, +} + +/// Perform batch and streaming aggregation with same input +/// and verify outputs of `AggregateExec` with pipeline breaking stream `GroupedHashAggregateStream` +/// and non-pipeline breaking stream `BoundedAggregateStream` produces same result. +async fn run_aggregate_test_internal( + test_data: Vec, + left_aggr_ctx: AggrTestContext, + right_aggr_ctx: AggrTestContext, + extra_checks: C, + group_by_columns: Vec<&str>, +) where + C: Fn(&[RecordBatch], &[RecordBatch]), +{ + let schema = test_data[0].schema(); + let aggregate_expr = vec![ AggregateExprBuilder::new(sum_udaf(), vec![col("d", &schema).unwrap()]) @@ -117,42 +167,44 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str .collect::>(); let group_by = PhysicalGroupBy::new_single(expr); - let aggregate_exec_running = Arc::new( + let aggregate_exec_usual = Arc::new( AggregateExec::try_new( AggregateMode::Partial, group_by.clone(), aggregate_expr.clone(), vec![None], - running_source, + left_aggr_ctx.data_source_exec.clone(), schema.clone(), ) .unwrap(), ) as Arc; - let aggregate_exec_usual = Arc::new( + let aggregate_exec_running = Arc::new( AggregateExec::try_new( AggregateMode::Partial, group_by.clone(), aggregate_expr.clone(), vec![None], - usual_source, + right_aggr_ctx.data_source_exec.clone(), schema.clone(), ) .unwrap(), ) as Arc; - let task_ctx = ctx.task_ctx(); - let collected_usual = collect(aggregate_exec_usual.clone(), task_ctx.clone()) - .await - .unwrap(); + let collected_usual = + collect(aggregate_exec_usual.clone(), left_aggr_ctx.task_ctx.clone()) + .await + .unwrap(); + + let collected_running = collect( + aggregate_exec_running.clone(), + right_aggr_ctx.task_ctx.clone(), + ) + .await + .unwrap(); + + extra_checks(&collected_usual, &collected_running); - let collected_running = collect(aggregate_exec_running.clone(), task_ctx.clone()) - .await - .unwrap(); - assert!(collected_running.len() > 2); - // Running should produce more chunk than the usual AggregateExec. - // Otherwise it means that we cannot generate result in running mode. - assert!(collected_running.len() > collected_usual.len()); // compare let usual_formatted = pretty_format_batches(&collected_usual).unwrap().to_string(); let running_formatted = pretty_format_batches(&collected_running) @@ -187,7 +239,7 @@ async fn run_aggregate_test(input1: Vec, group_by_columns: Vec<&str displayable(aggregate_exec_running.as_ref()).indent(false), usual_formatted, running_formatted, - pretty_format_batches(&input1).unwrap(), + pretty_format_batches(&test_data).unwrap(), ); } } @@ -311,6 +363,7 @@ async fn group_by_string_test( let actual = extract_result_counts(results); assert_eq!(expected, actual); } + async fn verify_ordered_aggregate(frame: &DataFrame, expected_sort: bool) { struct Visitor { expected_sort: bool, From 6a4cf5b6bc7d8a3d45b685d8335fbc8cf2ebf23f Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 03:49:07 +0800 Subject: [PATCH 085/107] add simple fuzz test for blocked approach. --- .../core/tests/fuzz_cases/aggregate_fuzz.rs | 130 +++++++++++++++++- 1 file changed, 126 insertions(+), 4 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index c9932f442c21..b7dda4e20bf9 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -33,16 +33,20 @@ use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::{collect, displayable, ExecutionPlan}; use datafusion::prelude::{DataFrame, SessionConfig, SessionContext}; use datafusion_common::tree_node::{TreeNode, TreeNodeRecursion, TreeNodeVisitor}; +use datafusion_common::ScalarValue; +use datafusion_execution::disk_manager::DiskManagerConfig; +use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_execution::TaskContext; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_plan::InputOrderMode; +use rand::seq::SliceRandom; use test_utils::{add_empty_batches, StringBatchGenerator}; use hashbrown::HashMap; use rand::rngs::StdRng; -use rand::{Rng, SeedableRng}; +use rand::{thread_rng, Rng, SeedableRng}; use tokio::task::JoinSet; /// Tests that streaming aggregate and batch (non streaming) aggregate produce @@ -78,6 +82,52 @@ async fn streaming_aggregate_test() { } } +/// Tests that streaming aggregate and batch (non streaming) aggregate produce +/// same results +#[tokio::test(flavor = "multi_thread")] +async fn blocked_approach_aggregate_test() { + let test_cases = [ + vec!["a"], + vec!["b", "a"], + vec!["c", "a"], + vec!["c", "b", "a"], + vec!["d", "a"], + vec!["d", "b", "a"], + vec!["d", "c", "a"], + vec!["d", "c", "b", "a"], + ]; + + let n_batch_size = 10; + let mut rng = thread_rng(); + let mut all_batch_sizes = (1..=50_usize).into_iter().collect::>(); + all_batch_sizes.shuffle(&mut rng); + let batch_sizes = &all_batch_sizes[0..n_batch_size]; + + let n = 300; + let distincts = vec![10, 20]; + for distinct in distincts { + let mut join_set = JoinSet::new(); + for batch_size in batch_sizes { + for i in 0..n { + let test_idx = i % test_cases.len(); + let group_by_columns = test_cases[test_idx].clone(); + join_set.spawn(run_blocked_approach_aggregate_test( + make_staggered_batches::(1000, distinct, i as u64), + group_by_columns, + *batch_size, + )); + } + } + while let Some(join_handle) = join_set.join_next().await { + // propagate errors + join_handle.unwrap(); + } + } +} + +/// Perform batch and streaming aggregation with same input +/// and verify outputs of `AggregateExec` with pipeline breaking stream `GroupedHashAggregateStream` +/// and non-pipeline breaking stream `BoundedAggregateStream` produces same result. async fn run_streaming_aggregate_test( test_data: Vec, group_by_columns: Vec<&str>, @@ -134,14 +184,86 @@ async fn run_streaming_aggregate_test( .await; } +/// Perform batch and blocked approach aggregations, and then verify their outputs. +async fn run_blocked_approach_aggregate_test( + test_data: Vec, + group_by_columns: Vec<&str>, + batch_size: usize, +) { + let schema = test_data[0].schema(); + + // Define test data source exec + let concat_input_record = concat_batches(&schema, &test_data).unwrap(); + let usual_source = Arc::new( + MemoryExec::try_new(&[vec![concat_input_record]], schema.clone(), None).unwrap(), + ); + + let running_source = Arc::new( + MemoryExec::try_new(&[test_data.clone()], schema.clone(), None).unwrap(), + ); + + // Define test task ctx + // Usual task ctx + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.batch_size", + ScalarValue::UInt64(Some(batch_size as u64)), + ); + let usual_ctx = Arc::new(TaskContext::default().with_session_config(session_config)); + + // Running task ctx + let mut session_config = SessionConfig::default(); + session_config = session_config.set( + "datafusion.execution.enable_aggregation_group_states_blocked_approach", + ScalarValue::Boolean(Some(true)), + ); + session_config = session_config.set( + "datafusion.execution.batch_size", + ScalarValue::UInt64(Some(batch_size as u64)), + ); + + let runtime = Arc::new( + RuntimeEnv::new( + RuntimeConfig::default().with_disk_manager(DiskManagerConfig::Disabled), + ) + .unwrap(), + ); + let running_ctx = Arc::new( + TaskContext::default() + .with_session_config(session_config) + .with_runtime(runtime), + ); + + // Run and check + let usual_aggr_ctx = AggrTestContext { + data_source_exec: usual_source, + task_ctx: usual_ctx, + }; + + let running_aggr_ctx = AggrTestContext { + data_source_exec: running_source, + task_ctx: running_ctx, + }; + + run_aggregate_test_internal( + test_data, + usual_aggr_ctx, + running_aggr_ctx, + |_, _| {}, + group_by_columns, + ) + .await; +} + +/// Options of the fuzz aggregation tests struct AggrTestContext { data_source_exec: Arc, task_ctx: Arc, } -/// Perform batch and streaming aggregation with same input -/// and verify outputs of `AggregateExec` with pipeline breaking stream `GroupedHashAggregateStream` -/// and non-pipeline breaking stream `BoundedAggregateStream` produces same result. +/// The internal test function for performing normal aggregation +/// and other optimized aggregations (without any optimizations, +/// e.g. streaming, blocked approach), and verify outputs of them. async fn run_aggregate_test_internal( test_data: Vec, left_aggr_ctx: AggrTestContext, From 551690d08a2191924b031d86755138841e3d8d34 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 04:05:39 +0800 Subject: [PATCH 086/107] don't support the `blocked approach` in bench until it compatible with spilling soon. --- benchmarks/src/clickbench.rs | 6 ------ benchmarks/src/tpch/run.rs | 7 ------- benchmarks/src/util/options.rs | 5 ----- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- 4 files changed, 1 insertion(+), 19 deletions(-) diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index fa6b7ffafeb7..a0f051d17623 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -123,12 +123,6 @@ impl RunOpt { .parquet .schema_force_string_view = self.common.string_view; - config - .options_mut() - .execution - .enable_aggregation_group_states_blocked_approach = - self.common.blocked_approach; - let ctx = SessionContext::new_with_config(config); self.register_hits(&ctx).await?; diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index ba1af56d05d3..ebc5ac0dbd5a 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -125,11 +125,6 @@ impl RunOpt { .execution .parquet .schema_force_string_view = self.common.string_view; - config - .options_mut() - .execution - .enable_aggregation_group_states_blocked_approach = - self.common.blocked_approach; let ctx = SessionContext::new_with_config(config); // register tables @@ -350,7 +345,6 @@ mod tests { batch_size: 8192, debug: false, string_view: false, - blocked_approach: false, }; let opt = RunOpt { query: Some(query), @@ -385,7 +379,6 @@ mod tests { batch_size: 8192, debug: false, string_view: false, - blocked_approach: false, }; let opt = RunOpt { query: Some(query), diff --git a/benchmarks/src/util/options.rs b/benchmarks/src/util/options.rs index af7fc6ef87d2..02591e293272 100644 --- a/benchmarks/src/util/options.rs +++ b/benchmarks/src/util/options.rs @@ -42,11 +42,6 @@ pub struct CommonOpt { /// when reading ParquetFiles #[structopt(long)] pub string_view: bool, - - /// If true, will use blocked approach to manage group values and states - /// in accumulators in aggregation. - #[structopt(long)] - pub blocked_approach: bool, } impl CommonOpt { diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index 72845a08b008..cd3bbfcde78b 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -99,7 +99,7 @@ async fn blocked_approach_aggregate_test() { let n_batch_size = 10; let mut rng = thread_rng(); - let mut all_batch_sizes = (1..=50_usize).into_iter().collect::>(); + let mut all_batch_sizes = (1..=50_usize).collect::>(); all_batch_sizes.shuffle(&mut rng); let batch_sizes = &all_batch_sizes[0..n_batch_size]; From 4a97d3516efeb33560280a2bb4fdb6b15e66e239 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 21:21:27 +0800 Subject: [PATCH 087/107] fix clippy. --- .../src/aggregate/groups_accumulator/prim_op.rs | 1 - 1 file changed, 1 deletion(-) 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 2ac2d9871027..0788ab619556 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 @@ -16,7 +16,6 @@ // under the License. use std::sync::Arc; -use std::usize; use arrow::array::{ArrayRef, AsArray, BooleanArray, PrimitiveArray}; use arrow::buffer::NullBuffer; From 189b4c35b747c2a89d064b4d786639afb765ea68 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 21:42:14 +0800 Subject: [PATCH 088/107] merge main and fix compile. --- datafusion/physical-plan/src/aggregates/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 0c6f67ba8de6..70b754319905 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1213,7 +1213,9 @@ mod tests { use datafusion_execution::config::SessionConfig; use datafusion_execution::disk_manager::DiskManagerConfig; use datafusion_execution::memory_pool::FairSpillPool; - use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_execution::runtime_env::{ + RuntimeConfig, RuntimeEnv, RuntimeEnvBuilder, + }; use datafusion_functions_aggregate::array_agg::array_agg_udaf; use datafusion_functions_aggregate::average::avg_udaf; use datafusion_functions_aggregate::count::count_udaf; From 11870cb2fdc3d26de5e93a80e9d0581a8a6d2429 Mon Sep 17 00:00:00 2001 From: kamille Date: Wed, 28 Aug 2024 23:50:51 +0800 Subject: [PATCH 089/107] fix clippy. --- .../src/aggregate/groups_accumulator/accumulate.rs | 2 -- 1 file changed, 2 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 b248192b6cad..835b4df93e0e 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -19,8 +19,6 @@ //! //! [`GroupsAccumulator`]: datafusion_expr_common::groups_accumulator::GroupsAccumulator -use std::usize; - use arrow::array::{Array, BooleanArray, BooleanBufferBuilder, PrimitiveArray}; use arrow::buffer::{BooleanBuffer, NullBuffer}; use arrow::datatypes::ArrowPrimitiveType; From 6ecd81b5f35805a883762f8553350c0a5528fb4c Mon Sep 17 00:00:00 2001 From: kamille Date: Thu, 29 Aug 2024 00:28:52 +0800 Subject: [PATCH 090/107] fix clippy. --- .../src/aggregate/groups_accumulator.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index a80b4bbf5f72..f96fef92d39c 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -29,7 +29,6 @@ use std::{ fmt::{self, Debug}, iter, ops::{Index, IndexMut}, - usize, }; use arrow::{ From 36e0791f8dd8350cf6866b55b97f985bd0ae3ab1 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 30 Aug 2024 01:20:19 +0800 Subject: [PATCH 091/107] add comments to architecture about blocked approach. --- .../physical-plan/src/aggregates/row_hash.rs | 42 ++++++++++++++++++- 1 file changed, 41 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 713906714100..41939ce93e82 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -274,7 +274,26 @@ impl SkipAggregationProbe { /// The accumulator state is not managed by this operator (e.g in the /// hash table). /// +/// An important optimization for [`group_values`] and [`accumulators`] +/// is to manage values using the blocked approach. +/// +/// In the original method, 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. +/// +/// 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 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 mode can be found in +/// [`maybe_enable_blocked_group_states`]. +/// /// [`group_values`]: Self::group_values +/// [`accumulators`]: Self::accumulators +/// [#7065]: https://github.com/apache/datafusion/issues/7065 /// /// # Partial Aggregate and multi-phase grouping /// @@ -344,6 +363,24 @@ impl SkipAggregationProbe { /// │ 2 │ 2 │ 3.0 │ │ 2 │ 2 │ 3.0 │ └────────────┘ /// └─────────────────┘ └─────────────────┘ /// ``` +/// +/// # Partial Aggregate and multi-phase grouping +/// +/// As described on [`Accumulator::state`], this operator is used in the context +/// "multi-phase" grouping when the mode is [`AggregateMode::Partial`]. +/// +/// An important optimization for multi-phase partial aggregation is to skip +/// partial aggregation when it is not effective enough to warrant the memory or +/// CPU cost, as is often the case for queries many distinct groups (high +/// cardinality group by). Memory is particularly important because each Partial +/// aggregator must store the intermediate state for each group. +/// +/// If the ratio of the number of groups to the number of input rows exceeds a +/// threshold, and [`GroupsAccumulator::supports_convert_to_state`] is +/// supported, this operator will stop applying Partial aggregation and directly +/// pass the input rows to the next aggregation phase. +/// +/// [`Accumulator::state`]: datafusion_expr::Accumulator::state pub(crate) struct GroupedHashAggregateStream { // ======================================================================== // PROPERTIES: @@ -603,7 +640,10 @@ impl GroupedHashAggregateStream { /// - 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` and all `GroupsAccumulator`s support blocked mode +/// - [`GroupValues::support_blocked_mode`] and all [`GroupsAccumulator::supports_blocked_mode`] are true +/// +/// [`GroupValues::supports_blocked_mode`]: crate::aggregates::group_values::GroupValues::supports_blocked_mode +/// [`GroupsAccumulator::supports_blocked_mode`]: datafusion_expr::GroupsAccumulator::supports_blocked_mode // TODO: support blocked optimization in streaming, spilling, and maybe empty accumulators case? fn maybe_enable_blocked_group_states( context: &TaskContext, From 44263077bfaf0ced031fb70fc40125ca28be776e Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 30 Aug 2024 01:26:01 +0800 Subject: [PATCH 092/107] fix comments. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 41939ce93e82..bf28f7f3b8f2 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -282,9 +282,10 @@ impl SkipAggregationProbe { /// 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 instead of -/// expanding the current one and copying the data. +/// 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]. /// From 886bb20f0ecc6aceba5c3e3baf81c401f13c523e Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 30 Aug 2024 01:27:20 +0800 Subject: [PATCH 093/107] fix typo. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index bf28f7f3b8f2..02a44209f18b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -105,7 +105,7 @@ struct SpillState { // ======================================================================== // STATES: // Fields changes during execution. Can be buffer, or state flags that - // influence the exeuction in parent `GroupedHashAggregateStream` + // influence the execution in parent `GroupedHashAggregateStream` // ======================================================================== /// If data has previously been spilled, the locations of the /// spill files (in Arrow IPC format) From c2cb573e048c151169a730adafb932905b1f7166 Mon Sep 17 00:00:00 2001 From: kamille Date: Fri, 30 Aug 2024 11:26:10 +0800 Subject: [PATCH 094/107] fix docs. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 02a44209f18b..91325fc9bb79 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -282,9 +282,9 @@ impl SkipAggregationProbe { /// copies, resulting in poor performance. /// /// In contrast, the blocked approach allocates capacity for the block -/// based on a predefined block size firstly. +/// 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) +/// (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]. @@ -641,10 +641,11 @@ impl GroupedHashAggregateStream { /// - 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::support_blocked_mode`] and all [`GroupsAccumulator::supports_blocked_mode`] are true +/// - [`GroupValues::supports_blocked_mode`] and all [`GroupsAccumulator::supports_blocked_mode`] are true /// /// [`GroupValues::supports_blocked_mode`]: crate::aggregates::group_values::GroupValues::supports_blocked_mode /// [`GroupsAccumulator::supports_blocked_mode`]: datafusion_expr::GroupsAccumulator::supports_blocked_mode +/// // TODO: support blocked optimization in streaming, spilling, and maybe empty accumulators case? fn maybe_enable_blocked_group_states( context: &TaskContext, From ef91012ede9bb1d520cb05276dcad2c5858ed229 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 13:22:36 +0800 Subject: [PATCH 095/107] a unified and low cost way to compute the different type `BlockedGroupIndex`s. --- .../src/aggregate/groups_accumulator.rs | 65 ++++--- .../groups_accumulator/accumulate.rs | 59 +++--- datafusion/functions-aggregate/src/count.rs | 78 +++----- .../src/aggregates/group_values/row.rs | 173 ++++++++---------- 4 files changed, 166 insertions(+), 209 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index f96fef92d39c..c7ab342b95dd 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -44,6 +44,10 @@ use datafusion_expr_common::accumulator::Accumulator; use datafusion_expr_common::groups_accumulator::{EmitTo, GroupsAccumulator}; pub const MAX_PREALLOC_BLOCK_SIZE: usize = 8192; +const FLAT_GROUP_INDEX_ID_MASK: u64 = 0; +const FLAT_GROUP_INDEX_OFFSET_MASK: u64 = u64::MAX; +const BLOCKED_GROUP_INDEX_ID_MASK: u64 = 0xffffffff00000000; +const BLOCKED_GROUP_INDEX_OFFSET_MASK: u64 = 0x00000000ffffffff; /// An adapter that implements [`GroupsAccumulator`] for any [`Accumulator`] /// @@ -452,37 +456,14 @@ impl EmitToExt for EmitTo { pub struct BlockedGroupIndex { pub block_id: u32, pub block_offset: u64, - pub is_blocked: bool, } impl BlockedGroupIndex { #[inline] - pub fn new_from_parts(block_id: u32, block_offset: u64, is_blocked: bool) -> Self { + pub fn new_from_parts(block_id: u32, block_offset: u64) -> Self { Self { block_id, block_offset, - is_blocked, - } - } - - #[inline] - pub fn new_flat(raw_index: usize) -> Self { - Self { - block_id: 0, - block_offset: raw_index as u64, - is_blocked: false, - } - } - - #[inline] - pub fn new_blocked(raw_index: usize) -> Self { - let block_id = ((raw_index as u64 >> 32) & 0x00000000ffffffff) as u32; - let block_offset = (raw_index as u64) & 0x00000000ffffffff; - - Self { - block_id, - block_offset, - is_blocked: true, } } @@ -496,11 +477,41 @@ impl BlockedGroupIndex { self.block_offset as usize } + #[inline] pub fn as_packed_index(&self) -> usize { - if self.is_blocked { - (((self.block_id as u64) << 32) | self.block_offset) as usize + (((self.block_id as u64) << 32) | self.block_offset) as usize + } +} + +pub struct BlockedGroupIndexBuilder { + block_id_mask: u64, + block_offset_mask: u64, +} + +impl BlockedGroupIndexBuilder { + #[inline] + pub fn new(is_blocked: bool) -> Self { + if is_blocked { + Self { + block_id_mask: BLOCKED_GROUP_INDEX_ID_MASK, + block_offset_mask: BLOCKED_GROUP_INDEX_OFFSET_MASK, + } } else { - self.block_offset as usize + Self { + block_id_mask: FLAT_GROUP_INDEX_ID_MASK, + block_offset_mask: FLAT_GROUP_INDEX_OFFSET_MASK, + } + } + } + + #[inline] + pub fn build(&self, packed_index: usize) -> BlockedGroupIndex { + let block_id = (((packed_index as u64) & self.block_id_mask) >> 32) as u32; + let block_offset = (packed_index as u64) & self.block_offset_mask; + + BlockedGroupIndex { + block_id, + block_offset, } } } 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 835b4df93e0e..770a10454880 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator/accumulate.rs @@ -26,7 +26,7 @@ use arrow::datatypes::ArrowPrimitiveType; use datafusion_expr_common::groups_accumulator::EmitTo; use crate::aggregate::groups_accumulator::{ - BlockedGroupIndex, Blocks, MAX_PREALLOC_BLOCK_SIZE, + BlockedGroupIndex, BlockedGroupIndexBuilder, Blocks, MAX_PREALLOC_BLOCK_SIZE, }; /// Track the accumulator null state per row: if any values for that @@ -303,32 +303,20 @@ impl BlockedNullState { false, ); let seen_values_blocks = &mut self.seen_values_blocks; + let group_index_builder = + BlockedGroupIndexBuilder::new(self.block_size.is_some()); - if self.block_size.is_some() { - do_blocked_accumulate( - group_indices, - values, - opt_filter, - BlockedGroupIndex::new_blocked, - value_fn, - |index: &BlockedGroupIndex| { - seen_values_blocks[index.block_id()] - .set_bit(index.block_offset(), true); - }, - ) - } else { - do_blocked_accumulate( - group_indices, - values, - opt_filter, - BlockedGroupIndex::new_flat, - value_fn, - |index: &BlockedGroupIndex| { - seen_values_blocks[index.block_id()] - .set_bit(index.block_offset(), true); - }, - ); - } + do_blocked_accumulate( + group_indices, + values, + opt_filter, + &group_index_builder, + value_fn, + |group_index| { + seen_values_blocks[group_index.block_id()] + .set_bit(group_index.block_offset(), true); + }, + ) } /// Similar as [NullState::build] but support the blocked version accumulator @@ -598,16 +586,15 @@ pub fn accumulate_indices( } } -fn do_blocked_accumulate( +fn do_blocked_accumulate( group_indices: &[usize], values: &PrimitiveArray, opt_filter: Option<&BooleanArray>, - group_index_parse_fn: G, + group_index_builder: &BlockedGroupIndexBuilder, mut value_fn: F1, mut set_valid_fn: F2, ) where T: ArrowPrimitiveType + Send, - G: Fn(usize) -> BlockedGroupIndex, F1: FnMut(&BlockedGroupIndex, T::Native) + Send, F2: FnMut(&BlockedGroupIndex) + Send, { @@ -617,7 +604,7 @@ fn do_blocked_accumulate( (false, None) => { let iter = group_indices.iter().zip(data.iter()); for (&group_index, &new_value) in iter { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = group_index_builder.build(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -645,7 +632,8 @@ fn do_blocked_accumulate( // valid bit was set, real value let is_valid = (mask & index_mask) != 0; if is_valid { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = + group_index_builder.build(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -663,7 +651,7 @@ fn do_blocked_accumulate( .for_each(|(i, (&group_index, &new_value))| { let is_valid = remainder_bits & (1 << i) != 0; if is_valid { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = group_index_builder.build(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -681,7 +669,7 @@ fn do_blocked_accumulate( .zip(filter.iter()) .for_each(|((&group_index, &new_value), filter_value)| { if let Some(true) = filter_value { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = group_index_builder.build(group_index); set_valid_fn(&blocked_index); value_fn(&blocked_index, new_value); } @@ -700,9 +688,9 @@ fn do_blocked_accumulate( .for_each(|((filter_value, &group_index), new_value)| { if let Some(true) = filter_value { if let Some(new_value) = new_value { - let blocked_index = group_index_parse_fn(group_index); + let blocked_index = group_index_builder.build(group_index); set_valid_fn(&blocked_index); - value_fn(&blocked_index, new_value) + value_fn(&blocked_index, new_value); } } }) @@ -933,7 +921,6 @@ mod test { BlockedGroupIndex::new_from_parts( block_id as u32, block_offset as u64, - true, ) .as_packed_index() }) diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index d9cd3988129c..e6d44b71ec6b 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -18,7 +18,7 @@ use ahash::RandomState; use datafusion_functions_aggregate_common::aggregate::count_distinct::BytesViewDistinctCountAccumulator; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - ensure_enough_room_for_values, BlockedGroupIndex, Blocks, EmitToExt, VecBlocks, + ensure_enough_room_for_values, BlockedGroupIndexBuilder, Blocks, EmitToExt, VecBlocks, }; use std::collections::HashSet; use std::ops::BitAnd; @@ -399,31 +399,19 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - if self.block_size.is_some() { - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - let blocked_index = BlockedGroupIndex::new_blocked(group_index); - let count = &mut self.counts[blocked_index.block_id()] - [blocked_index.block_offset()]; - *count += 1; - }, - ); - } else { - accumulate_indices( - group_indices, - values.logical_nulls().as_ref(), - opt_filter, - |group_index| { - let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = &mut self.counts[blocked_index.block_id()] - [blocked_index.block_offset()]; - *count += 1; - }, - ); - } + let group_index_builder = + BlockedGroupIndexBuilder::new(self.block_size.is_some()); + accumulate_indices( + group_indices, + values.logical_nulls().as_ref(), + opt_filter, + |group_index| { + let blocked_index = group_index_builder.build(group_index); + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; + *count += 1; + }, + ); Ok(()) } @@ -450,31 +438,19 @@ impl GroupsAccumulator for CountGroupsAccumulator { 0, ); - if self.block_size.is_some() { - do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new_blocked(group_index); - let count = &mut self.counts[blocked_index.block_id()] - [blocked_index.block_offset()]; - *count += partial_count; - }, - ); - } else { - do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let blocked_index = BlockedGroupIndex::new_flat(group_index); - let count = &mut self.counts[blocked_index.block_id()] - [blocked_index.block_offset()]; - *count += partial_count; - }, - ); - } + let group_index_builder = + BlockedGroupIndexBuilder::new(self.block_size.is_some()); + do_count_merge_batch( + values, + group_indices, + opt_filter, + |group_index, partial_count| { + let blocked_index = group_index_builder.build(group_index); + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; + *count += partial_count; + }, + ); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/group_values/row.rs b/datafusion/physical-plan/src/aggregates/group_values/row.rs index ea55ac5d450d..2ff433ac7a87 100644 --- a/datafusion/physical-plan/src/aggregates/group_values/row.rs +++ b/datafusion/physical-plan/src/aggregates/group_values/row.rs @@ -29,7 +29,7 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::proxy::{RawTableAllocExt, VecAllocExt}; use datafusion_expr::EmitTo; use datafusion_functions_aggregate_common::aggregate::groups_accumulator::{ - BlockedGroupIndex, Blocks, + BlockedGroupIndex, BlockedGroupIndexBuilder, Blocks, }; use hashbrown::raw::RawTable; @@ -135,79 +135,68 @@ impl GroupValues for GroupValuesRows { batch_hashes.resize(n_rows, 0); create_hashes(cols, &self.random_state, batch_hashes)?; - let mut get_or_create_groups = - |group_index_parse_fn: fn(usize) -> BlockedGroupIndex| { - for (row, &target_hash) in batch_hashes.iter().enumerate() { - let entry = - self.map.get_mut(target_hash, |(exist_hash, group_idx)| { - // Somewhat surprisingly, this closure can be called even if the - // hash doesn't match, so check the hash first with an integer - // comparison first avoid the more expensive comparison with - // group value. https://github.com/apache/datafusion/pull/11718 - if target_hash != *exist_hash { - return false; - } - - // verify that the group that we are inserting with hash is - // actually the same key value as the group in - // existing_idx (aka group_values @ row) - let blocked_index = group_index_parse_fn(*group_idx); - group_rows.row(row) - == group_values[blocked_index.block_id()] - .row(blocked_index.block_offset()) - }); - - let group_idx = match entry { - // Existing group_index for this group value - Some((_hash, group_idx)) => *group_idx, - // 1.2 Need to create new entry for the group - None => { - // Add new entry to aggr_state and save newly created index - if let Some(blk_size) = self.block_size { - if group_values.current().unwrap().num_rows() == blk_size - { - // Use blk_size as offset cap, - // and old block's buffer size as buffer cap - let new_buf_cap = - rows_buffer_size(group_values.current().unwrap()); - let new_blk = self - .row_converter - .empty_rows(blk_size, new_buf_cap); - group_values.push_block(new_blk); - } - } + let group_index_builder = + BlockedGroupIndexBuilder::new(self.block_size.is_some()); + for (row, &target_hash) in batch_hashes.iter().enumerate() { + let entry = self.map.get_mut(target_hash, |(exist_hash, group_idx)| { + // Somewhat surprisingly, this closure can be called even if the + // hash doesn't match, so check the hash first with an integer + // comparison first avoid the more expensive comparison with + // group value. https://github.com/apache/datafusion/pull/11718 + if target_hash != *exist_hash { + return false; + } - let blk_id = group_values.num_blocks() - 1; - let cur_blk = group_values.current_mut().unwrap(); - let blk_offset = cur_blk.num_rows(); - cur_blk.push(group_rows.row(row)); - - let blocked_index = BlockedGroupIndex::new_from_parts( - blk_id as u32, - blk_offset as u64, - self.block_size.is_some(), - ); - let group_idx = blocked_index.as_packed_index(); - - // for hasher function, use precomputed hash value - self.map.insert_accounted( - (target_hash, group_idx), - |(hash, _group_index)| *hash, - &mut self.map_size, - ); - - group_idx + // verify that the group that we are inserting with hash is + // actually the same key value as the group in + // existing_idx (aka group_values @ row) + let blocked_index = group_index_builder.build(*group_idx); + group_rows.row(row) + == group_values[blocked_index.block_id()] + .row(blocked_index.block_offset()) + }); + + let group_idx = match entry { + // Existing group_index for this group value + Some((_hash, group_idx)) => *group_idx, + // 1.2 Need to create new entry for the group + None => { + // Add new entry to aggr_state and save newly created index + if let Some(blk_size) = self.block_size { + if group_values.current().unwrap().num_rows() == blk_size { + // Use blk_size as offset cap, + // and old block's buffer size as buffer cap + let new_buf_cap = + rows_buffer_size(group_values.current().unwrap()); + let new_blk = + self.row_converter.empty_rows(blk_size, new_buf_cap); + group_values.push_block(new_blk); } - }; - groups.push(group_idx); + } + + let blk_id = group_values.num_blocks() - 1; + let cur_blk = group_values.current_mut().unwrap(); + let blk_offset = cur_blk.num_rows(); + cur_blk.push(group_rows.row(row)); + + let blocked_index = BlockedGroupIndex::new_from_parts( + blk_id as u32, + blk_offset as u64, + ); + let group_idx = blocked_index.as_packed_index(); + + // for hasher function, use precomputed hash value + self.map.insert_accounted( + (target_hash, group_idx), + |(hash, _group_index)| *hash, + &mut self.map_size, + ); + + group_idx } }; - - if self.block_size.is_some() { - get_or_create_groups(BlockedGroupIndex::new_blocked); - } else { - get_or_create_groups(BlockedGroupIndex::new_flat); - }; + groups.push(group_idx); + } self.group_values = group_values; @@ -294,33 +283,27 @@ impl GroupValues for GroupValuesRows { let cur_blk = group_values.pop_first_block().unwrap(); let output = self.row_converter.convert_rows(cur_blk.iter())?; - let mut shift_down_values = - |group_index_parse_fn: fn(usize) -> BlockedGroupIndex| unsafe { - for bucket in self.map.iter() { - // Decrement group index by n - let group_idx = bucket.as_ref().1; - let old_blk_idx = group_index_parse_fn(group_idx); - match old_blk_idx.block_id().checked_sub(1) { - // Group index was >= n, shift value down - Some(new_blk_id) => { - let new_group_idx = BlockedGroupIndex::new_from_parts( - new_blk_id as u32, - old_blk_idx.block_offset, - true, - ); - bucket.as_mut().1 = new_group_idx.as_packed_index(); - } - // Group index was < n, so remove from table - None => self.map.erase(bucket), + unsafe { + let group_index_builder = + BlockedGroupIndexBuilder::new(self.block_size.is_some()); + for bucket in self.map.iter() { + // Decrement group index by n + let group_idx = bucket.as_ref().1; + let old_blk_idx = group_index_builder.build(group_idx); + match old_blk_idx.block_id().checked_sub(1) { + // Group index was >= n, shift value down + Some(new_blk_id) => { + let new_group_idx = BlockedGroupIndex::new_from_parts( + new_blk_id as u32, + old_blk_idx.block_offset, + ); + bucket.as_mut().1 = new_group_idx.as_packed_index(); } + // Group index was < n, so remove from table + None => self.map.erase(bucket), } - }; - - if self.block_size.is_some() { - shift_down_values(BlockedGroupIndex::new_blocked); - } else { - shift_down_values(BlockedGroupIndex::new_flat); - }; + } + } output } From 31356d86aceb5181625020d8ee7206e62113664d Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 16:02:15 +0800 Subject: [PATCH 096/107] add test to `BlockedGroupIndexBuilder`. --- .../src/aggregate/groups_accumulator.rs | 33 ++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index c7ab342b95dd..6675ec6955ed 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -452,7 +452,7 @@ impl EmitToExt for EmitTo { /// - `block_id` is always 0 /// - total 64 bits used to represent the `block offset` /// -#[derive(Debug, Clone, Copy)] +#[derive(Debug, Clone, Copy, PartialEq, Eq)] pub struct BlockedGroupIndex { pub block_id: u32, pub block_offset: u64, @@ -813,6 +813,8 @@ pub fn ensure_enough_room_for_values( #[cfg(test)] mod tests { + use std::u32; + use super::*; #[test] @@ -955,4 +957,33 @@ mod tests { let actual = emit.take_needed_from_blocks(&mut values); assert!(actual.is_empty()); } + + #[test] + fn test_blocked_group_index_build() { + let group_index1 = (0_u64 << 32) | 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 5d2ac01945ce1d4d61c34b8a70ade59cc5331ebd Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 16:22:02 +0800 Subject: [PATCH 097/107] add more inlines. --- .../src/aggregate/groups_accumulator.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 6675ec6955ed..a7995991f6b1 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -549,10 +549,12 @@ impl Blocks { } } + #[inline] pub fn current(&self) -> Option<&T> { self.current.as_ref() } + #[inline] pub fn current_mut(&mut self) -> Option<&mut T> { self.current.as_mut() } From 0a7b52bc06cb71e5d5beaf506fa5382aa9c4bd67 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 19:57:34 +0800 Subject: [PATCH 098/107] fix clippy. --- .../src/aggregate/groups_accumulator.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index a7995991f6b1..7d64436083a4 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -962,7 +962,7 @@ mod tests { #[test] fn test_blocked_group_index_build() { - let group_index1 = (0_u64 << 32) | 1; + let group_index1 = 1; let group_index2 = (42_u64 << 32) | 2; let group_index3 = ((u32::MAX as u64) << 32) | 3; From 318c6500261c9b4dfe6dacd7cbd0aebf11f0eed6 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 20:11:50 +0800 Subject: [PATCH 099/107] improve config comments. --- datafusion/common/src/config.rs | 14 +- .../test_files/information_schema.slt | 2 +- docs/source/user-guide/configs.md | 176 +++++++++--------- 3 files changed, 96 insertions(+), 96 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index ee5cc2695a54..047b80629053 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -344,14 +344,14 @@ config_namespace! { /// Should DataFusion use the the blocked approach to manage the groups /// values and their related states in accumulators. By default, the single - /// approach will be used, and such group values and states will be managed - /// using a single big block(can think a `Vec`), obviously as the block growing up, - /// many copies will be triggered and finally get a bad performance. + /// approach will be used, 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. /// If setting this flag to `true`, the blocked approach will be used. - /// We will allocate the `block size` capacity for block first, and when we - /// found the block has been filled to `block size` limit, we will allocate - /// next block rather than growing current block and copying the data. This - /// approach can eliminate all unnecessary copies and get a good performance finally. + /// 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. /// We plan to make this the default in the future when tests are enough. pub enable_aggregation_group_states_blocked_approach: bool, default = false } diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 2353756dee69..16485fadf624 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -264,7 +264,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold f 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_group_states_blocked_approach false Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, and such group values and states will be managed using a single big block(can think a `Vec`), obviously as the block growing up, many copies will be triggered and finally get a bad performance. If setting this flag to `true`, the blocked approach will be used. We will allocate the `block size` capacity for block first, and when we found the block has been filled to `block size` limit, we will allocate next block rather than growing current block and copying the data. This approach can eliminate all unnecessary copies and get a good performance finally. We plan to make this the default in the future when tests are enough. +datafusion.execution.enable_aggregation_group_states_blocked_approach false Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. datafusion.execution.enable_recursive_ctes true Should DataFusion support recursive CTEs datafusion.execution.keep_partition_by_columns false Should DataFusion keep the columns used for partition_by in the output RecordBatches 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`). diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 17a0aac3bc85..36044b84e3f6 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,91 +35,91 @@ 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.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.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 | -| 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 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| 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.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.enable_aggregation_group_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, and such group values and states will be managed using a single big block(can think a `Vec`), obviously as the block growing up, many copies will be triggered and finally get a bad performance. If setting this flag to `true`, the blocked approach will be used. We will allocate the `block size` capacity for block first, and when we found the block has been filled to `block size` limit, we will allocate next block rather than growing current block and copying the data. This approach can eliminate all unnecessary copies and get a good performance finally. We plan to make this the default in the future when tests are enough. | -| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| 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, and Ansi. | -| 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. | +| 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.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.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 | +| 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 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | +| 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.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.enable_aggregation_group_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. | +| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| 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, and Ansi. | +| 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. | From 3d820948db37647ca81e8ccf6e1c1b5e36e471c8 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 21:50:19 +0800 Subject: [PATCH 100/107] remove deprecated function. --- .../src/aggregate/groups_accumulator.rs | 2 - datafusion/functions-aggregate/src/count.rs | 77 +++++++------------ 2 files changed, 29 insertions(+), 50 deletions(-) diff --git a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs index 7d64436083a4..9387ef583e2b 100644 --- a/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs +++ b/datafusion/functions-aggregate-common/src/aggregate/groups_accumulator.rs @@ -815,8 +815,6 @@ pub fn ensure_enough_room_for_values( #[cfg(test)] mod tests { - use std::u32; - use super::*; #[test] diff --git a/datafusion/functions-aggregate/src/count.rs b/datafusion/functions-aggregate/src/count.rs index e6d44b71ec6b..24a8c09671a5 100644 --- a/datafusion/functions-aggregate/src/count.rs +++ b/datafusion/functions-aggregate/src/count.rs @@ -428,7 +428,12 @@ impl GroupsAccumulator for CountGroupsAccumulator { } assert_eq!(values.len(), 1, "one argument to merge_batch"); - let values = &values[0]; + // first batch is counts, second is partial sums + let partial_counts = values[0].as_primitive::(); + + // intermediate counts are always created as non null + assert_eq!(partial_counts.null_count(), 0); + let partial_counts = partial_counts.values(); // Adds the counts with the partial counts ensure_enough_room_for_values( @@ -440,17 +445,29 @@ impl GroupsAccumulator for CountGroupsAccumulator { let group_index_builder = BlockedGroupIndexBuilder::new(self.block_size.is_some()); - do_count_merge_batch( - values, - group_indices, - opt_filter, - |group_index, partial_count| { - let blocked_index = group_index_builder.build(group_index); - let count = &mut self.counts[blocked_index.block_id()] - [blocked_index.block_offset()]; - *count += partial_count; - }, - ); + + match opt_filter { + Some(filter) => filter + .iter() + .zip(group_indices.iter()) + .zip(partial_counts.iter()) + .for_each(|((filter_value, &group_index), partial_count)| { + if let Some(true) = filter_value { + let blocked_index = group_index_builder.build(group_index); + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; + *count += partial_count; + } + }), + None => group_indices.iter().zip(partial_counts.iter()).for_each( + |(&group_index, partial_count)| { + let blocked_index = group_index_builder.build(group_index); + let count = &mut self.counts[blocked_index.block_id()] + [blocked_index.block_offset()]; + *count += partial_count; + }, + ), + } Ok(()) } @@ -574,42 +591,6 @@ fn null_count_for_multiple_cols(values: &[ArrayRef]) -> usize { } } -/// The intermediate states merging function in count accumulator. -/// It can support blocked and flat mode count accumulator through -/// different `update_group_fn`s. -fn do_count_merge_batch( - values: &ArrayRef, - group_indices: &[usize], - opt_filter: Option<&BooleanArray>, - mut update_group_fn: F, -) where - F: FnMut(usize, i64), -{ - // first batch is counts, second is partial sums - let partial_counts = values.as_primitive::(); - - // intermediate counts are always created as non null - assert_eq!(partial_counts.null_count(), 0); - let partial_counts = partial_counts.values(); - - match opt_filter { - Some(filter) => filter - .iter() - .zip(group_indices.iter()) - .zip(partial_counts.iter()) - .for_each(|((filter_value, &group_index), &partial_count)| { - if let Some(true) = filter_value { - update_group_fn(group_index, partial_count); - } - }), - None => group_indices.iter().zip(partial_counts.iter()).for_each( - |(&group_index, &partial_count)| { - update_group_fn(group_index, partial_count); - }, - ), - } -} - /// General purpose distinct accumulator that works for any DataType by using /// [`ScalarValue`]. /// From b7a443a16420a5fb400297d05ee59c2122c9de6f Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 22:19:01 +0800 Subject: [PATCH 101/107] improve docs. --- .../physical-plan/src/aggregates/row_hash.rs | 57 +++++++------------ 1 file changed, 20 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 91325fc9bb79..edb15ae3cb81 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -274,28 +274,6 @@ impl SkipAggregationProbe { /// The accumulator state is not managed by this operator (e.g in the /// hash table). /// -/// An important optimization for [`group_values`] and [`accumulators`] -/// is to manage values using the blocked approach. -/// -/// In the original method, 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. -/// -/// 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 mode can be found in -/// [`maybe_enable_blocked_group_states`]. -/// -/// [`group_values`]: Self::group_values -/// [`accumulators`]: Self::accumulators -/// [#7065]: https://github.com/apache/datafusion/issues/7065 -/// /// # Partial Aggregate and multi-phase grouping /// /// As described on [`Accumulator::state`], this operator is used in the context @@ -364,24 +342,29 @@ impl SkipAggregationProbe { /// │ 2 │ 2 │ 3.0 │ │ 2 │ 2 │ 3.0 │ └────────────┘ /// └─────────────────┘ └─────────────────┘ /// ``` +/// +/// # Blocked approach for intermediate values +/// An important optimization for [`group_values`] and [`accumulators`] +/// is to manage values using the blocked approach. /// -/// # Partial Aggregate and multi-phase grouping -/// -/// As described on [`Accumulator::state`], this operator is used in the context -/// "multi-phase" grouping when the mode is [`AggregateMode::Partial`]. -/// -/// An important optimization for multi-phase partial aggregation is to skip -/// partial aggregation when it is not effective enough to warrant the memory or -/// CPU cost, as is often the case for queries many distinct groups (high -/// cardinality group by). Memory is particularly important because each Partial -/// aggregator must store the intermediate state for each group. +/// In the original method, 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. /// -/// If the ratio of the number of groups to the number of input rows exceeds a -/// threshold, and [`GroupsAccumulator::supports_convert_to_state`] is -/// supported, this operator will stop applying Partial aggregation and directly -/// pass the input rows to the next aggregation phase. +/// 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]. /// -/// [`Accumulator::state`]: datafusion_expr::Accumulator::state +/// The conditions that trigger the blocked mode can be found in +/// [`maybe_enable_blocked_group_states`]. +/// +/// [`group_values`]: Self::group_values +/// [`accumulators`]: Self::accumulators +/// [#7065]: https://github.com/apache/datafusion/issues/7065 pub(crate) struct GroupedHashAggregateStream { // ======================================================================== // PROPERTIES: From 0cff3be43ab8c9529f2df92dfd0b4d61470cac50 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 22:26:03 +0800 Subject: [PATCH 102/107] rename the on/off option to enable_aggregation_intermediate_states_blocked_approach. --- datafusion/common/src/config.rs | 2 +- .../physical-plan/src/aggregates/row_hash.rs | 16 +- .../test_files/information_schema.slt | 4 +- docs/source/user-guide/configs.md | 176 +++++++++--------- 4 files changed, 99 insertions(+), 99 deletions(-) diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 047b80629053..2f49681df798 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -353,7 +353,7 @@ config_namespace! { /// we allocate a new block (also with the same predefined block size based capacity) // instead of expanding the current one and copying the data. /// We plan to make this the default in the future when tests are enough. - pub enable_aggregation_group_states_blocked_approach: bool, default = false + pub enable_aggregation_intermediate_states_blocked_approach: bool, default = false } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index edb15ae3cb81..cf1ea882798b 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -450,7 +450,7 @@ pub(crate) struct GroupedHashAggregateStream { skip_aggregation_probe: Option, /// Have we enabled the blocked optimization for group values and accumulators. - enable_blocked_group_states: bool, + is_blocked_approach_on: bool, // ======================================================================== // EXECUTION RESOURCES: @@ -613,7 +613,7 @@ impl GroupedHashAggregateStream { spill_state, group_values_soft_limit: agg.limit, skip_aggregation_probe, - enable_blocked_group_states, + is_blocked_approach_on: enable_blocked_group_states, }) } } @@ -641,7 +641,7 @@ fn maybe_enable_blocked_group_states( .session_config() .options() .execution - .enable_aggregation_group_states_blocked_approach + .enable_aggregation_intermediate_states_blocked_approach || !matches!(group_ordering, GroupOrdering::None) || accumulators.is_empty() || enable_spilling(context.runtime_env().disk_manager.as_ref()) @@ -1082,7 +1082,7 @@ impl GroupedHashAggregateStream { && matches!(self.mode, AggregateMode::Partial) && self.update_memory_reservation().is_err() { - if !self.enable_blocked_group_states { + if !self.is_blocked_approach_on { let n = self.group_values.len() / self.batch_size * self.batch_size; let batch = self.emit(EmitTo::First(n), false)?; self.exec_state = ExecutionState::ProducingOutput(batch); @@ -1130,12 +1130,12 @@ impl GroupedHashAggregateStream { // We should disable the blocked optimization for `GroupValues` and `GroupAccumulator`s here, // because the blocked mode can't support `Emit::First(exact n)` which is needed in // streaming aggregation. - if self.enable_blocked_group_states { + if self.is_blocked_approach_on { self.group_values.alter_block_size(None)?; self.accumulators .iter_mut() .try_for_each(|acc| acc.alter_block_size(None))?; - self.enable_blocked_group_states = false; + self.is_blocked_approach_on = false; } self.input_done = false; @@ -1159,7 +1159,7 @@ 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() { - if !self.enable_blocked_group_states { + if !self.is_blocked_approach_on { let batch = self.emit(EmitTo::All, false)?; ExecutionState::ProducingOutput(batch) } else { @@ -1196,7 +1196,7 @@ 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 !self.enable_blocked_group_states { + if !self.is_blocked_approach_on { let batch = self.emit(EmitTo::All, false)?; self.exec_state = ExecutionState::ProducingOutput(batch); } else { diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 16485fadf624..565cea4ec0d0 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -173,7 +173,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 datafusion.execution.batch_size 8192 datafusion.execution.coalesce_batches true datafusion.execution.collect_statistics false -datafusion.execution.enable_aggregation_group_states_blocked_approach false +datafusion.execution.enable_aggregation_intermediate_states_blocked_approach false datafusion.execution.enable_recursive_ctes true datafusion.execution.keep_partition_by_columns false datafusion.execution.listing_table_ignore_subdirectory true @@ -264,7 +264,7 @@ datafusion.execution.aggregate.scalar_update_factor 10 Specifies the threshold f 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_group_states_blocked_approach false Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. +datafusion.execution.enable_aggregation_intermediate_states_blocked_approach false Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. datafusion.execution.enable_recursive_ctes true Should DataFusion support recursive CTEs datafusion.execution.keep_partition_by_columns false Should DataFusion keep the columns used for partition_by in the output RecordBatches 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`). diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 36044b84e3f6..bc2eed2c14f7 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,91 +35,91 @@ 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.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.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 | -| 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 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | -| 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.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.enable_aggregation_group_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. | -| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| 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, and Ansi. | -| 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. | +| 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.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.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 | +| 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 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| datafusion.execution.aggregate.scalar_update_factor | 10 | Specifies the threshold for using `ScalarValue`s to update accumulators during high-cardinality aggregations for each input batch. The aggregation is considered high-cardinality if the number of affected groups is greater than or equal to `batch_size / scalar_update_factor`. In such cases, `ScalarValue`s are utilized for updating accumulators, rather than the default batch-slice approach. This can lead to performance improvements. By adjusting the `scalar_update_factor`, you can balance the trade-off between more efficient accumulator updates and the number of groups affected. | +| 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.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.enable_aggregation_intermediate_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. | +| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| 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, and Ansi. | +| 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. | From a2d81a59cac598dbf0d7a237d5d7b71b7149a82a Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 22:29:55 +0800 Subject: [PATCH 103/107] fix doc. --- datafusion/physical-plan/src/aggregates/row_hash.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index cf1ea882798b..7f8f1a7d6ff9 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -344,6 +344,7 @@ impl SkipAggregationProbe { /// ``` /// /// # Blocked approach for intermediate values +/// /// An important optimization for [`group_values`] and [`accumulators`] /// is to manage values using the blocked approach. /// @@ -365,6 +366,7 @@ impl SkipAggregationProbe { /// [`group_values`]: Self::group_values /// [`accumulators`]: Self::accumulators /// [#7065]: https://github.com/apache/datafusion/issues/7065 +/// pub(crate) struct GroupedHashAggregateStream { // ======================================================================== // PROPERTIES: From 1db8633dc593479f701bdd0918595552ef54a773 Mon Sep 17 00:00:00 2001 From: kamille Date: Sun, 1 Sep 2024 23:01:00 +0800 Subject: [PATCH 104/107] fix fmt and tests. --- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 2 +- datafusion/physical-plan/src/aggregates/mod.rs | 2 +- datafusion/physical-plan/src/aggregates/row_hash.rs | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index cd3bbfcde78b..f98170631d3f 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -214,7 +214,7 @@ async fn run_blocked_approach_aggregate_test( // Running task ctx let mut session_config = SessionConfig::default(); session_config = session_config.set( - "datafusion.execution.enable_aggregation_group_states_blocked_approach", + "datafusion.execution.enable_aggregation_intermediate_states_blocked_approach", ScalarValue::Boolean(Some(true)), ); session_config = session_config.set( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 70b754319905..612eff7966a5 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2580,7 +2580,7 @@ mod tests { // Define task context let mut session_config = SessionConfig::default(); session_config = session_config.set( - "datafusion.execution.enable_aggregation_group_states_blocked_approach", + "datafusion.execution.enable_aggregation_intermediate_states_blocked_approach", ScalarValue::Boolean(Some(true)), ); session_config = session_config.set( diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 7f8f1a7d6ff9..b811dc79f260 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -342,9 +342,9 @@ impl SkipAggregationProbe { /// │ 2 │ 2 │ 3.0 │ │ 2 │ 2 │ 3.0 │ └────────────┘ /// └─────────────────┘ └─────────────────┘ /// ``` -/// +/// /// # Blocked approach for intermediate values -/// +/// /// An important optimization for [`group_values`] and [`accumulators`] /// is to manage values using the blocked approach. /// @@ -366,7 +366,7 @@ impl SkipAggregationProbe { /// [`group_values`]: Self::group_values /// [`accumulators`]: Self::accumulators /// [#7065]: https://github.com/apache/datafusion/issues/7065 -/// +/// pub(crate) struct GroupedHashAggregateStream { // ======================================================================== // PROPERTIES: @@ -622,7 +622,7 @@ impl GroupedHashAggregateStream { /// Check if we can enable the blocked optimization for `GroupValues` and `GroupsAccumulator`s. /// The blocked optimization will be enabled when: -/// - When `enable_aggregation_group_states_blocked_approach` is true +/// - 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) From 6613288fb5fbf927078620c2dc0edbb3d5319115 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 3 Sep 2024 09:23:43 +0800 Subject: [PATCH 105/107] update docs. --- docs/source/user-guide/configs.md | 175 +++++++++++++++--------------- 1 file changed, 87 insertions(+), 88 deletions(-) diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 1ad43adba389..6a23d31f4180 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -35,91 +35,90 @@ 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.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.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 | -| 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 41.0.0 | (writing) Sets "created by" property | -| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | -| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | -| 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.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.enable_aggregation_intermediate_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. | -| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | -| 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, and Ansi. | -| 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. | - +| 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.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.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 | +| 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 41.0.0 | (writing) Sets "created by" property | +| datafusion.execution.parquet.column_index_truncate_length | 64 | (writing) Sets column index truncate length | +| 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.parquet.schema_force_string_view | false | (reading) If true, parquet reader will read columns of `Utf8/Utf8Large` with `Utf8View`, and `Binary/BinaryLarge` with `BinaryView`. | +| 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.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.enable_aggregation_intermediate_states_blocked_approach | false | Should DataFusion use the the blocked approach to manage the groups values and their related states in accumulators. By default, the single approach will be used, 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. If setting this flag to `true`, 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) We plan to make this the default in the future when tests are enough. | +| 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.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.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 | true | When set to true, SQL parser will normalize options value (convert value to lowercase) | +| 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, and Ansi. | +| 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. | From cbafbc53a025cb23394df6a2d2ef6f88b1665eb1 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 3 Sep 2024 09:26:16 +0800 Subject: [PATCH 106/107] fix fmt. --- .../physical-plan/src/aggregates/mod.rs | 86 +++++++++---------- 1 file changed, 43 insertions(+), 43 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f832238f02cc..bbe6037f5e19 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2620,48 +2620,48 @@ mod tests { assert_batches_eq!(expected, &output); } - #[test] - fn group_exprs_nullable() -> Result<()> { - let input_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Float32, false), - Field::new("b", DataType::Float32, false), - ])); - - let aggr_expr = vec![AggregateExprBuilder::new( - count_udaf(), - vec![col("a", &input_schema)?], - ) - .schema(Arc::clone(&input_schema)) - .alias("COUNT(a)") - .build()?]; + #[test] + fn group_exprs_nullable() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Float32, false), + Field::new("b", DataType::Float32, false), + ])); - let grouping_set = PhysicalGroupBy { - expr: vec![ - (col("a", &input_schema)?, "a".to_string()), - (col("b", &input_schema)?, "b".to_string()), - ], - null_expr: vec![ - (lit(ScalarValue::Float32(None)), "a".to_string()), - (lit(ScalarValue::Float32(None)), "b".to_string()), - ], - groups: vec![ - vec![false, true], // (a, NULL) - vec![false, false], // (a,b) - ], - }; - let aggr_schema = create_schema( - &input_schema, - &grouping_set.expr, - &aggr_expr, - grouping_set.exprs_nullable(), - AggregateMode::Final, - )?; - let expected_schema = Schema::new(vec![ - Field::new("a", DataType::Float32, false), - Field::new("b", DataType::Float32, true), - Field::new("COUNT(a)", DataType::Int64, false), - ]); - assert_eq!(aggr_schema, expected_schema); - Ok(()) - } + let aggr_expr = + vec![ + AggregateExprBuilder::new(count_udaf(), vec![col("a", &input_schema)?]) + .schema(Arc::clone(&input_schema)) + .alias("COUNT(a)") + .build()?, + ]; + + let grouping_set = PhysicalGroupBy { + expr: vec![ + (col("a", &input_schema)?, "a".to_string()), + (col("b", &input_schema)?, "b".to_string()), + ], + null_expr: vec![ + (lit(ScalarValue::Float32(None)), "a".to_string()), + (lit(ScalarValue::Float32(None)), "b".to_string()), + ], + groups: vec![ + vec![false, true], // (a, NULL) + vec![false, false], // (a,b) + ], + }; + let aggr_schema = create_schema( + &input_schema, + &grouping_set.expr, + &aggr_expr, + grouping_set.exprs_nullable(), + AggregateMode::Final, + )?; + let expected_schema = Schema::new(vec![ + Field::new("a", DataType::Float32, false), + Field::new("b", DataType::Float32, true), + Field::new("COUNT(a)", DataType::Int64, false), + ]); + assert_eq!(aggr_schema, expected_schema); + Ok(()) + } } From d258ea98081f67e2ce497a7e1ffaaeaf1d5c6ef0 Mon Sep 17 00:00:00 2001 From: kamille Date: Tue, 3 Sep 2024 19:51:16 +0800 Subject: [PATCH 107/107] fix compile. --- datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs | 6 +++--- datafusion/physical-plan/src/aggregates/mod.rs | 6 ++++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs index f98170631d3f..ce89ad363cf4 100644 --- a/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/aggregate_fuzz.rs @@ -207,7 +207,7 @@ async fn run_blocked_approach_aggregate_test( let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.batch_size", - ScalarValue::UInt64(Some(batch_size as u64)), + &ScalarValue::UInt64(Some(batch_size as u64)), ); let usual_ctx = Arc::new(TaskContext::default().with_session_config(session_config)); @@ -215,11 +215,11 @@ async fn run_blocked_approach_aggregate_test( let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.enable_aggregation_intermediate_states_blocked_approach", - ScalarValue::Boolean(Some(true)), + &ScalarValue::Boolean(Some(true)), ); session_config = session_config.set( "datafusion.execution.batch_size", - ScalarValue::UInt64(Some(batch_size as u64)), + &ScalarValue::UInt64(Some(batch_size as u64)), ); let runtime = Arc::new( diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index bbe6037f5e19..426ca91eadec 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -2585,11 +2585,11 @@ mod tests { let mut session_config = SessionConfig::default(); session_config = session_config.set( "datafusion.execution.enable_aggregation_intermediate_states_blocked_approach", - ScalarValue::Boolean(Some(true)), + &ScalarValue::Boolean(Some(true)), ); session_config = session_config.set( "datafusion.execution.batch_size", - ScalarValue::UInt64(Some(1)), + &ScalarValue::UInt64(Some(1)), ); let runtime = Arc::new( @@ -2618,6 +2618,8 @@ mod tests { "+----+----+--------+", ]; assert_batches_eq!(expected, &output); + + Ok(()) } #[test]