diff --git a/Cargo.lock b/Cargo.lock index d7daf8cdd012..84731ebaa917 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5083,7 +5083,6 @@ dependencies = [ "sha2", "socket2", "sqlx", - "strength_reduce", "sysinfo", "temp-env", "tempfile", diff --git a/src/query/expression/src/kernels/group_by.rs b/src/query/expression/src/kernels/group_by.rs index ebef511ae4f4..fac1a64a8054 100644 --- a/src/query/expression/src/kernels/group_by.rs +++ b/src/query/expression/src/kernels/group_by.rs @@ -23,16 +23,11 @@ use super::group_by_hash::HashMethodSerializer; use super::group_by_hash::HashMethodSingleBinary; use crate::types::DataType; use crate::DataBlock; -use crate::HashMethodDictionarySerializer; use crate::HashMethodKeysU128; use crate::HashMethodKeysU256; impl DataBlock { - pub fn choose_hash_method( - chunk: &DataBlock, - indices: &[usize], - efficiently_memory: bool, - ) -> Result { + pub fn choose_hash_method(chunk: &DataBlock, indices: &[usize]) -> Result { let hash_key_types = indices .iter() .map(|&offset| { @@ -42,13 +37,10 @@ impl DataBlock { .collect::>>(); let hash_key_types = hash_key_types?; - Self::choose_hash_method_with_types(&hash_key_types, efficiently_memory) + Self::choose_hash_method_with_types(&hash_key_types) } - pub fn choose_hash_method_with_types( - hash_key_types: &[DataType], - efficiently_memory: bool, - ) -> Result { + pub fn choose_hash_method_with_types(hash_key_types: &[DataType]) -> Result { if hash_key_types.len() == 1 && matches!( hash_key_types[0], @@ -74,14 +66,8 @@ impl DataBlock { if hash_key_type.is_nullable() { group_key_len += 1; } - } else if !efficiently_memory || hash_key_types.len() == 1 { - return Ok(HashMethodKind::Serializer(HashMethodSerializer::default())); } else { - return Ok(HashMethodKind::DictionarySerializer( - HashMethodDictionarySerializer { - dict_keys: hash_key_types.len(), - }, - )); + return Ok(HashMethodKind::Serializer(HashMethodSerializer::default())); } } diff --git a/src/query/expression/src/kernels/group_by_hash/method.rs b/src/query/expression/src/kernels/group_by_hash/method.rs index 4dfb1af8b5be..2e69c1b73ce3 100644 --- a/src/query/expression/src/kernels/group_by_hash/method.rs +++ b/src/query/expression/src/kernels/group_by_hash/method.rs @@ -31,7 +31,6 @@ use crate::types::DecimalDataType; use crate::types::NumberDataType; use crate::types::StringColumn; use crate::Column; -use crate::HashMethodDictionarySerializer; use crate::HashMethodKeysU128; use crate::HashMethodKeysU16; use crate::HashMethodKeysU256; @@ -91,7 +90,6 @@ pub trait HashMethod: Clone + Sync + Send + 'static { #[derive(Clone, Debug)] pub enum HashMethodKind { Serializer(HashMethodSerializer), - DictionarySerializer(HashMethodDictionarySerializer), SingleBinary(HashMethodSingleBinary), KeysU8(HashMethodKeysU8), KeysU16(HashMethodKeysU16), @@ -106,7 +104,7 @@ macro_rules! with_hash_method { ( | $t:tt | $($tail:tt)* ) => { match_template::match_template! { $t = [Serializer, SingleBinary, KeysU8, KeysU16, - KeysU32, KeysU64, KeysU128, KeysU256, DictionarySerializer], + KeysU32, KeysU64, KeysU128, KeysU256], $($tail)* } } @@ -123,26 +121,6 @@ macro_rules! with_join_hash_method { } } -#[macro_export] -macro_rules! with_mappedhash_method { - ( | $t:tt | $($tail:tt)* ) => { - match_template::match_template! { - $t = [ - Serializer => HashMethodSerializer, - SingleBinary => HashMethodSingleBinary, - KeysU8 => HashMethodKeysU8, - KeysU16 => HashMethodKeysU16, - KeysU32 => HashMethodKeysU32, - KeysU64 => HashMethodKeysU64, - KeysU128 => HashMethodKeysU128, - KeysU256 => HashMethodKeysU256, - DictionarySerializer => HashMethodDictionarySerializer - ], - $($tail)* - } - } -} - impl HashMethodKind { pub fn name(&self) -> String { with_hash_method!(|T| match self { @@ -164,7 +142,6 @@ impl HashMethodKind { HashMethodKind::KeysU256(_) => { DataType::Decimal(DecimalDataType::Decimal256(i256::default_decimal_size())) } - HashMethodKind::DictionarySerializer(_) => DataType::Binary, } } } diff --git a/src/query/expression/src/kernels/group_by_hash/method_dict_serializer.rs b/src/query/expression/src/kernels/group_by_hash/method_dict_serializer.rs deleted file mode 100644 index cf0cdd91967b..000000000000 --- a/src/query/expression/src/kernels/group_by_hash/method_dict_serializer.rs +++ /dev/null @@ -1,142 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::ptr::NonNull; - -use databend_common_exception::Result; -use databend_common_hashtable::DictionaryKeys; -use databend_common_hashtable::FastHash; -use either::Either; - -use super::utils::serialize_group_columns; -use crate::Column; -use crate::HashMethod; -use crate::InputColumns; -use crate::KeyAccessor; -use crate::KeysState; - -#[derive(Debug, Clone, PartialEq, Eq)] -pub struct HashMethodDictionarySerializer { - pub dict_keys: usize, -} - -impl HashMethod for HashMethodDictionarySerializer { - type HashKey = DictionaryKeys; - type HashKeyIter<'a> = std::slice::Iter<'a, DictionaryKeys>; - - fn name(&self) -> String { - "DictionarySerializer".to_string() - } - - fn build_keys_state(&self, group_columns: InputColumns, num_rows: usize) -> Result { - // fixed type serialize one column to dictionary - let mut dictionary_columns = Vec::with_capacity(group_columns.len()); - let mut serialize_columns = Vec::new(); - for group_column in group_columns.iter() { - match group_column { - Column::Binary(v) | Column::Variant(v) | Column::Bitmap(v) => { - debug_assert_eq!(v.len(), num_rows); - dictionary_columns.push(Either::Right(v.clone())); - } - Column::String(v) => { - debug_assert_eq!(v.len(), num_rows); - dictionary_columns.push(Either::Left(v.clone())); - } - _ => serialize_columns.push(group_column.clone()), - } - } - - if !serialize_columns.is_empty() { - // The serialize_size is equal to the number of bytes required by serialization. - let mut serialize_size = 0; - for column in serialize_columns.iter() { - serialize_size += column.serialize_size(); - } - let state = - serialize_group_columns((&serialize_columns).into(), num_rows, serialize_size); - dictionary_columns.push(Either::Right(state)); - } - - let mut keys = Vec::with_capacity(num_rows * dictionary_columns.len()); - let mut points = Vec::with_capacity(num_rows * dictionary_columns.len()); - - for row in 0..num_rows { - let start = points.len(); - - for dictionary_column in &dictionary_columns { - let data = match dictionary_column { - Either::Left(l) => unsafe { l.index_unchecked(row).as_bytes() }, - Either::Right(r) => unsafe { r.index_unchecked(row) }, - }; - points.push(NonNull::from(data)); - } - - keys.push(DictionaryKeys::create(&points[start..])) - } - - Ok(KeysState::Dictionary { - dictionaries: keys, - keys_point: points, - columns: dictionary_columns, - }) - } - - fn build_keys_iter<'a>(&self, keys_state: &'a KeysState) -> Result> { - match keys_state { - KeysState::Dictionary { dictionaries, .. } => Ok(dictionaries.iter()), - _ => unreachable!(), - } - } - - fn build_keys_accessor( - &self, - keys_state: KeysState, - ) -> Result>> { - match keys_state { - KeysState::Dictionary { dictionaries, .. } => { - Ok(Box::new(DicKeyAccessor::new(dictionaries))) - } - _ => unreachable!(), - } - } - - fn build_keys_hashes(&self, keys_state: &KeysState, hashes: &mut Vec) { - match keys_state { - KeysState::Dictionary { dictionaries, .. } => { - hashes.extend(dictionaries.iter().map(|key| key.fast_hash())); - } - _ => unreachable!(), - } - } -} - -pub struct DicKeyAccessor { - data: Vec, -} - -impl DicKeyAccessor { - pub fn new(data: Vec) -> Self { - Self { data } - } -} - -impl KeyAccessor for DicKeyAccessor { - type Key = DictionaryKeys; - - /// # Safety - /// Calling this method with an out-of-bounds index is *[undefined behavior]*. - unsafe fn key_unchecked(&self, index: usize) -> &Self::Key { - self.data.get_unchecked(index) - } -} diff --git a/src/query/expression/src/kernels/group_by_hash/mod.rs b/src/query/expression/src/kernels/group_by_hash/mod.rs index d6c8cf8af051..d18b4aa1ea3e 100644 --- a/src/query/expression/src/kernels/group_by_hash/mod.rs +++ b/src/query/expression/src/kernels/group_by_hash/mod.rs @@ -13,14 +13,12 @@ // limitations under the License. mod method; -mod method_dict_serializer; mod method_fixed_keys; mod method_serializer; mod method_single_string; mod utils; pub use method::*; -pub use method_dict_serializer::*; pub use method_fixed_keys::*; pub use method_serializer::*; pub use method_single_string::*; diff --git a/src/query/expression/tests/it/group_by.rs b/src/query/expression/tests/it/group_by.rs index 735e9171f95a..88325dcbfa3b 100644 --- a/src/query/expression/tests/it/group_by.rs +++ b/src/query/expression/tests/it/group_by.rs @@ -36,10 +36,10 @@ fn test_group_by_hash() -> Result<()> { StringType::from_data(vec!["x1", "x1", "x2", "x1", "x2", "x3"]), ]); - let method = DataBlock::choose_hash_method(&block, &[0, 3], false)?; + let method = DataBlock::choose_hash_method(&block, &[0, 3])?; assert_eq!(method.name(), HashMethodSerializer::default().name(),); - let method = DataBlock::choose_hash_method(&block, &[0, 1, 2], false)?; + let method = DataBlock::choose_hash_method(&block, &[0, 1, 2])?; assert_eq!(method.name(), HashMethodKeysU32::default().name()); diff --git a/src/query/service/Cargo.toml b/src/query/service/Cargo.toml index 1d3b3a4acdfb..2985dcdedc20 100644 --- a/src/query/service/Cargo.toml +++ b/src/query/service/Cargo.toml @@ -164,7 +164,6 @@ serde_urlencoded = { workspace = true } sha2 = { workspace = true } socket2 = { workspace = true } sqlx = { workspace = true } -strength_reduce = { workspace = true } sysinfo = { workspace = true } tempfile = { workspace = true } tokio = { workspace = true } diff --git a/src/query/service/src/pipelines/builders/builder_aggregate.rs b/src/query/service/src/pipelines/builders/builder_aggregate.rs index 9c16033ae25d..2c2223efd2f2 100644 --- a/src/query/service/src/pipelines/builders/builder_aggregate.rs +++ b/src/query/service/src/pipelines/builders/builder_aggregate.rs @@ -16,12 +16,8 @@ use std::sync::Arc; use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; -use databend_common_expression::with_hash_method; -use databend_common_expression::with_mappedhash_method; use databend_common_expression::AggregateFunctionRef; -use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; -use databend_common_expression::HashMethodKind; use databend_common_expression::HashTableConfig; use databend_common_expression::LimitType; use databend_common_expression::SortColumnDescription; @@ -45,9 +41,7 @@ use crate::pipelines::processors::transforms::aggregator::FinalSingleStateAggreg use crate::pipelines::processors::transforms::aggregator::PartialSingleStateAggregator; use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillWriter; use crate::pipelines::processors::transforms::aggregator::TransformExpandGroupingSets; -use crate::pipelines::processors::transforms::aggregator::TransformGroupBySpillWriter; use crate::pipelines::processors::transforms::aggregator::TransformPartialAggregate; -use crate::pipelines::processors::transforms::aggregator::TransformPartialGroupBy; use crate::pipelines::PipelineBuilder; impl PipelineBuilder { @@ -124,12 +118,7 @@ impl PipelineBuilder { }); } - let efficiently_memory = self.settings.get_efficiently_memory_group_by()?; - - let group_cols = ¶ms.group_columns; let schema_before_group_by = params.input_schema.clone(); - let sample_block = DataBlock::empty_with_schema(schema_before_group_by.clone()); - let method = DataBlock::choose_hash_method(&sample_block, group_cols, efficiently_memory)?; // Need a global atomic to read the max current radix bits hint let partial_agg_config = if !self.is_exchange_neighbor { @@ -161,30 +150,13 @@ impl PipelineBuilder { } self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - match params.aggregate_functions.is_empty() { - true => with_mappedhash_method!(|T| match method.clone() { - HashMethodKind::T(method) => TransformPartialGroupBy::try_create( - self.ctx.clone(), - method, - input, - output, - params.clone(), - partial_agg_config.clone() - ), - }), - false => with_mappedhash_method!(|T| match method.clone() { - HashMethodKind::T(method) => TransformPartialAggregate::try_create( - self.ctx.clone(), - method, - input, - output, - params.clone(), - partial_agg_config.clone() - ), - }), - }?, - )) + Ok(ProcessorPtr::create(TransformPartialAggregate::try_create( + self.ctx.clone(), + input, + output, + params.clone(), + partial_agg_config.clone(), + )?)) })?; // If cluster mode, spill write will be completed in exchange serialize, because we need scatter the block data first @@ -193,45 +165,18 @@ impl PipelineBuilder { let location_prefix = query_spill_prefix(self.ctx.get_tenant().tenant_name(), &self.ctx.get_id()); self.main_pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - match params.aggregate_functions.is_empty() { - true => with_mappedhash_method!(|T| match method.clone() { - HashMethodKind::T(method) => TransformGroupBySpillWriter::create( - self.ctx.clone(), - input, - output, - method, - operator.clone(), - location_prefix.clone() - ), - }), - false => with_mappedhash_method!(|T| match method.clone() { - HashMethodKind::T(method) => TransformAggregateSpillWriter::create( - self.ctx.clone(), - input, - output, - method, - operator.clone(), - params.clone(), - location_prefix.clone() - ), - }), - }, - )) + Ok(ProcessorPtr::create(TransformAggregateSpillWriter::create( + self.ctx.clone(), + input, + output, + operator.clone(), + params.clone(), + location_prefix.clone(), + ))) })?; } - self.exchange_injector = match params.aggregate_functions.is_empty() { - true => with_mappedhash_method!(|T| match method.clone() { - HashMethodKind::T(method) => - AggregateInjector::<_, ()>::create(self.ctx.clone(), method, params.clone()), - }), - false => with_mappedhash_method!(|T| match method.clone() { - HashMethodKind::T(method) => - AggregateInjector::<_, usize>::create(self.ctx.clone(), method, params.clone()), - }), - }; - + self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); Ok(()) } @@ -264,48 +209,15 @@ impl PipelineBuilder { return Ok(()); } - let efficiently_memory = self.settings.get_efficiently_memory_group_by()?; - - let group_cols = ¶ms.group_columns; - let schema_before_group_by = params.input_schema.clone(); - let sample_block = DataBlock::empty_with_schema(schema_before_group_by); - let method = DataBlock::choose_hash_method(&sample_block, group_cols, efficiently_memory)?; - let old_inject = self.exchange_injector.clone(); - match params.aggregate_functions.is_empty() { - true => with_hash_method!(|T| match method { - HashMethodKind::T(v) => { - let input: &PhysicalPlan = &aggregate.input; - if matches!(input, PhysicalPlan::ExchangeSource(_)) { - self.exchange_injector = AggregateInjector::<_, ()>::create( - self.ctx.clone(), - v.clone(), - params.clone(), - ); - } - - self.build_pipeline(&aggregate.input)?; - self.exchange_injector = old_inject; - build_partition_bucket::<_, ()>(v, &mut self.main_pipeline, params.clone()) - } - }), - false => with_hash_method!(|T| match method { - HashMethodKind::T(v) => { - let input: &PhysicalPlan = &aggregate.input; - if matches!(input, PhysicalPlan::ExchangeSource(_)) { - self.exchange_injector = AggregateInjector::<_, usize>::create( - self.ctx.clone(), - v.clone(), - params.clone(), - ); - } - self.build_pipeline(&aggregate.input)?; - self.exchange_injector = old_inject; - build_partition_bucket::<_, usize>(v, &mut self.main_pipeline, params.clone()) - } - }), + let input: &PhysicalPlan = &aggregate.input; + if matches!(input, PhysicalPlan::ExchangeSource(_)) { + self.exchange_injector = AggregateInjector::create(self.ctx.clone(), params.clone()); } + self.build_pipeline(&aggregate.input)?; + self.exchange_injector = old_inject; + build_partition_bucket(&mut self.main_pipeline, params.clone()) } pub fn build_aggregator_params( diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_cell.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_cell.rs deleted file mode 100644 index 8e91237cdf30..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_cell.rs +++ /dev/null @@ -1,217 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::any::Any; -use std::marker::PhantomData; -use std::sync::Arc; - -use databend_common_base::runtime::drop_guard; -use databend_common_functions::aggregates::StateAddr; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; - -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::group_by::Area; -use crate::pipelines::processors::transforms::group_by::ArenaHolder; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; -use crate::pipelines::processors::transforms::group_by::PolymorphicKeysHelper; - -// Manage unsafe memory usage, free memory when the cell is destroyed. -pub struct HashTableCell { - pub hashtable: T::HashTable, - pub arena: Area, - pub arena_holders: Vec, - pub _dropper: Option>>, -} - -unsafe impl Send for HashTableCell {} - -unsafe impl Sync for HashTableCell {} - -impl Drop for HashTableCell { - fn drop(&mut self) { - drop_guard(move || { - if let Some(dropper) = self._dropper.take() { - dropper.destroy(&mut self.hashtable); - } - }) - } -} - -impl HashTableCell { - pub fn create( - inner: T::HashTable, - _dropper: Arc>, - ) -> HashTableCell { - HashTableCell:: { - hashtable: inner, - arena_holders: vec![], - _dropper: Some(_dropper), - arena: Area::create(), - } - } - - pub fn len(&self) -> usize { - self.hashtable.len() - } - - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - pub fn allocated_bytes(&self) -> usize { - self.hashtable.bytes_len(false) - + self.arena.allocated_bytes() - + self - .arena_holders - .iter() - .map(ArenaHolder::allocated_bytes) - .sum::() - } -} - -pub trait HashTableDropper { - fn as_any(&self) -> &dyn Any; - fn destroy(&self, hashtable: &mut T::HashTable); -} - -pub struct GroupByHashTableDropper { - _phantom: PhantomData, -} - -impl GroupByHashTableDropper { - pub fn create() -> Arc> { - Arc::new(GroupByHashTableDropper:: { - _phantom: Default::default(), - }) - } -} - -impl HashTableDropper for GroupByHashTableDropper { - fn as_any(&self) -> &dyn Any { - self - } - - fn destroy(&self, _: &mut T::HashTable<()>) { - // do nothing - } -} - -pub struct AggregateHashTableDropper { - params: Arc, - _phantom: PhantomData, -} - -impl AggregateHashTableDropper { - pub fn create(params: Arc) -> Arc> { - Arc::new(AggregateHashTableDropper:: { - params, - _phantom: Default::default(), - }) - } -} - -impl HashTableDropper for AggregateHashTableDropper { - fn as_any(&self) -> &dyn Any { - self - } - - fn destroy(&self, hashtable: &mut T::HashTable) { - let aggregator_params = self.params.as_ref(); - let aggregate_functions = &aggregator_params.aggregate_functions; - let offsets_aggregate_states = &aggregator_params.offsets_aggregate_states; - - let functions = aggregate_functions - .iter() - .filter(|p| p.need_manual_drop_state()) - .collect::>(); - - let state_offsets = offsets_aggregate_states - .iter() - .enumerate() - .filter(|(idx, _)| aggregate_functions[*idx].need_manual_drop_state()) - .map(|(_, s)| *s) - .collect::>(); - - if !state_offsets.is_empty() { - for group_entity in hashtable.iter() { - let place = Into::::into(*group_entity.get()); - - for (function, state_offset) in functions.iter().zip(state_offsets.iter()) { - unsafe { function.drop_state(place.next(*state_offset)) } - } - } - } - } -} - -pub struct PartitionedHashTableDropper { - _inner_dropper: Arc>, -} - -impl PartitionedHashTableDropper { - pub fn create( - _inner_dropper: Arc>, - ) -> Arc, V>> { - Arc::new(Self { _inner_dropper }) - } - - pub fn split_cell( - mut v: HashTableCell, V>, - ) -> Vec> { - unsafe { - let arena = std::mem::replace(&mut v.arena, Area::create()); - v.arena_holders.push(ArenaHolder::create(Some(arena))); - - let dropper = v - ._dropper - .as_ref() - .unwrap() - .as_any() - .downcast_ref::>() - .unwrap() - ._inner_dropper - .clone(); - - let mut cells = Vec::with_capacity(256); - while let Some(table) = v.hashtable.pop_first_inner_table() { - let mut table_cell = HashTableCell::create(table, dropper.clone()); - table_cell.arena_holders = v.arena_holders.to_vec(); - cells.push(table_cell); - } - - cells - } - } -} - -impl - HashTableDropper, V> for PartitionedHashTableDropper -{ - fn as_any(&self) -> &dyn Any { - self - } - - fn destroy( - &self, - hashtable: &mut as PolymorphicKeysHelper< - PartitionedHashMethod, - >>::HashTable, - ) { - for inner_table in hashtable.iter_tables_mut() { - self._inner_dropper.destroy(inner_table) - } - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs index dee1ca855ae4..83d5dd0f791f 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_exchange_injector.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::marker::PhantomData; use std::sync::Arc; use bumpalo::Bump; @@ -24,34 +23,19 @@ use databend_common_expression::DataBlock; use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::PayloadFlushState; -use databend_common_hashtable::FastHash; -use databend_common_hashtable::HashtableEntryMutRefLike; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::ProcessorPtr; use databend_common_pipeline_core::query_spill_prefix; use databend_common_pipeline_core::Pipeline; use databend_common_settings::FlightCompression; use databend_common_storage::DataOperator; -use strength_reduce::StrengthReducedU64; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::HashTablePayload; use crate::pipelines::processors::transforms::aggregator::serde::TransformExchangeAggregateSerializer; use crate::pipelines::processors::transforms::aggregator::serde::TransformExchangeAsyncBarrier; -use crate::pipelines::processors::transforms::aggregator::serde::TransformExchangeGroupBySerializer; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::HashTableCell; use crate::pipelines::processors::transforms::aggregator::TransformAggregateDeserializer; use crate::pipelines::processors::transforms::aggregator::TransformAggregateSerializer; use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillWriter; -use crate::pipelines::processors::transforms::aggregator::TransformGroupByDeserializer; -use crate::pipelines::processors::transforms::aggregator::TransformGroupBySerializer; -use crate::pipelines::processors::transforms::aggregator::TransformGroupBySpillWriter; -use crate::pipelines::processors::transforms::group_by::Area; -use crate::pipelines::processors::transforms::group_by::ArenaHolder; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::servers::flight::v1::exchange::DataExchange; use crate::servers::flight::v1::exchange::ExchangeInjector; use crate::servers::flight::v1::exchange::ExchangeSorting; @@ -60,95 +44,40 @@ use crate::servers::flight::v1::exchange::ShuffleExchangeParams; use crate::servers::flight::v1::scatter::FlightScatter; use crate::sessions::QueryContext; -struct AggregateExchangeSorting { - _phantom: PhantomData<(Method, V)>, -} +struct AggregateExchangeSorting {} pub fn compute_block_number(bucket: isize, max_partition_count: usize) -> Result { Ok(max_partition_count as isize * 1000 + bucket) } -impl ExchangeSorting - for AggregateExchangeSorting -{ +impl ExchangeSorting for AggregateExchangeSorting { fn block_number(&self, data_block: &DataBlock) -> Result { match data_block.get_meta() { None => Ok(-1), - Some(block_meta_info) => { - match AggregateMeta::::downcast_ref_from(block_meta_info) { - None => Err(ErrorCode::Internal(format!( - "Internal error, AggregateExchangeSorting only recv AggregateMeta {:?}", - serde_json::to_string(block_meta_info) - ))), - Some(meta_info) => match meta_info { - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::Serialized(v) => { - compute_block_number(v.bucket, v.max_partition_count) - } - AggregateMeta::HashTable(v) => Ok(v.bucket), - AggregateMeta::AggregatePayload(v) => { - compute_block_number(v.bucket, v.max_partition_count) - } - AggregateMeta::AggregateSpilling(_) - | AggregateMeta::Spilled(_) - | AggregateMeta::BucketSpilled(_) - | AggregateMeta::Spilling(_) => Ok(-1), - }, - } - } + Some(block_meta_info) => match AggregateMeta::downcast_ref_from(block_meta_info) { + None => Err(ErrorCode::Internal(format!( + "Internal error, AggregateExchangeSorting only recv AggregateMeta {:?}", + serde_json::to_string(block_meta_info) + ))), + Some(meta_info) => match meta_info { + AggregateMeta::Partitioned { .. } => unreachable!(), + AggregateMeta::Serialized(v) => { + compute_block_number(v.bucket, v.max_partition_count) + } + AggregateMeta::AggregatePayload(v) => { + compute_block_number(v.bucket, v.max_partition_count) + } + AggregateMeta::AggregateSpilling(_) + | AggregateMeta::Spilled(_) + | AggregateMeta::BucketSpilled(_) => Ok(-1), + }, + }, } } } -struct HashTableHashScatter { - method: Method, +struct HashTableHashScatter { buckets: usize, - _phantom: PhantomData, -} - -fn scatter( - mut payload: HashTablePayload, - buckets: usize, - method: &Method, -) -> Result>> { - let mut buckets = Vec::with_capacity(buckets); - - for _ in 0..buckets.capacity() { - buckets.push(method.create_hash_table(Arc::new(Bump::new()))?); - } - - let mods = StrengthReducedU64::new(buckets.len() as u64); - for item in payload.cell.hashtable.iter() { - let bucket_index = (item.key().fast_hash() % mods) as usize; - - unsafe { - match buckets[bucket_index].insert_and_entry(item.key()) { - Ok(mut entry) => { - *entry.get_mut() = *item.get(); - } - Err(mut entry) => { - *entry.get_mut() = *item.get(); - } - } - } - } - - let mut res = Vec::with_capacity(buckets.len()); - let dropper = payload.cell._dropper.take(); - let arena = std::mem::replace(&mut payload.cell.arena, Area::create()); - payload - .cell - .arena_holders - .push(ArenaHolder::create(Some(arena))); - - for bucket_table in buckets { - let mut cell = HashTableCell::::create(bucket_table, dropper.clone().unwrap()); - cell.arena_holders - .extend(payload.cell.arena_holders.clone()); - res.push(cell); - } - - Ok(res) } fn scatter_payload(mut payload: Payload, buckets: usize) -> Result> { @@ -234,49 +163,22 @@ fn scatter_partitioned_payload( Ok(buckets) } -impl FlightScatter - for HashTableHashScatter -{ +impl FlightScatter for HashTableHashScatter { fn execute(&self, mut data_block: DataBlock) -> Result> { if let Some(block_meta) = data_block.take_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_from(block_meta) { + if let Some(block_meta) = AggregateMeta::downcast_from(block_meta) { let mut blocks = Vec::with_capacity(self.buckets); match block_meta { AggregateMeta::Spilled(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::Spilling(payload) => { - let method = PartitionedHashMethod::create(self.method.clone()); - for hashtable_cell in scatter(payload, self.buckets, &method)? { - blocks.push(match hashtable_cell.hashtable.len() == 0 { - true => DataBlock::empty(), - false => DataBlock::empty_with_meta( - AggregateMeta::::create_spilling(hashtable_cell), - ), - }); - } - } AggregateMeta::AggregateSpilling(payload) => { for p in scatter_partitioned_payload(payload, self.buckets)? { blocks.push(match p.len() == 0 { true => DataBlock::empty(), false => DataBlock::empty_with_meta( - AggregateMeta::::create_agg_spilling(p), - ), - }); - } - } - AggregateMeta::HashTable(payload) => { - let bucket = payload.bucket; - for hashtable_cell in scatter(payload, self.buckets, &self.method)? { - blocks.push(match hashtable_cell.hashtable.len() == 0 { - true => DataBlock::empty(), - false => DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable( - bucket, - hashtable_cell, - ), + AggregateMeta::create_agg_spilling(p), ), }); } @@ -285,13 +187,13 @@ impl FlightScatter for payload in scatter_payload(p.payload, self.buckets)? { blocks.push(match payload.len() == 0 { true => DataBlock::empty(), - false => DataBlock::empty_with_meta( - AggregateMeta::::create_agg_payload( + false => { + DataBlock::empty_with_meta(AggregateMeta::create_agg_payload( p.bucket, payload, p.max_partition_count, - ), - ), + )) + } }); } } @@ -307,34 +209,27 @@ impl FlightScatter } } -pub struct AggregateInjector { +pub struct AggregateInjector { ctx: Arc, - method: Method, tenant: String, aggregator_params: Arc, - _phantom: PhantomData, } -impl AggregateInjector { +impl AggregateInjector { pub fn create( ctx: Arc, - method: Method, params: Arc, ) -> Arc { let tenant = ctx.get_tenant(); - Arc::new(AggregateInjector:: { + Arc::new(AggregateInjector { ctx, - method, tenant: tenant.tenant_name().to_string(), aggregator_params: params, - _phantom: Default::default(), }) } } -impl ExchangeInjector - for AggregateInjector -{ +impl ExchangeInjector for AggregateInjector { fn flight_scatter( &self, _: &Arc, @@ -344,19 +239,15 @@ impl ExchangeInjector DataExchange::Merge(_) => unreachable!(), DataExchange::Broadcast(_) => unreachable!(), DataExchange::ShuffleDataExchange(exchange) => { - Ok(Arc::new(Box::new(HashTableHashScatter:: { - method: self.method.clone(), + Ok(Arc::new(Box::new(HashTableHashScatter { buckets: exchange.destination_ids.len(), - _phantom: Default::default(), }))) } } } fn exchange_sorting(&self) -> Option> { - Some(Arc::new(AggregateExchangeSorting:: { - _phantom: Default::default(), - })) + Some(Arc::new(AggregateExchangeSorting {})) } fn apply_merge_serializer( @@ -365,47 +256,25 @@ impl ExchangeInjector _compression: Option, pipeline: &mut Pipeline, ) -> Result<()> { - let method = &self.method; let params = self.aggregator_params.clone(); let operator = DataOperator::instance().operator(); let location_prefix = query_spill_prefix(&self.tenant, &self.ctx.get_id()); pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - match params.aggregate_functions.is_empty() { - true => TransformGroupBySpillWriter::create( - self.ctx.clone(), - input, - output, - method.clone(), - operator.clone(), - location_prefix.clone(), - ), - false => TransformAggregateSpillWriter::create( - self.ctx.clone(), - input, - output, - method.clone(), - operator.clone(), - params.clone(), - location_prefix.clone(), - ), - }, - )) + Ok(ProcessorPtr::create(TransformAggregateSpillWriter::create( + self.ctx.clone(), + input, + output, + operator.clone(), + params.clone(), + location_prefix.clone(), + ))) })?; - pipeline.add_transform( - |input, output| match params.aggregate_functions.is_empty() { - true => TransformGroupBySerializer::try_create(input, output, method.clone()), - false => TransformAggregateSerializer::try_create( - input, - output, - method.clone(), - params.clone(), - ), - }, - ) + pipeline.add_transform(|input, output| { + TransformAggregateSerializer::try_create(input, output, params.clone()) + }) } fn apply_shuffle_serializer( @@ -414,7 +283,6 @@ impl ExchangeInjector compression: Option, pipeline: &mut Pipeline, ) -> Result<()> { - let method = &self.method; let params = self.aggregator_params.clone(); let operator = DataOperator::instance().operator(); let location_prefix = query_spill_prefix(&self.tenant, &self.ctx.get_id()); @@ -429,31 +297,17 @@ impl ExchangeInjector pipeline.add_transform(|input, output| { Ok(ProcessorPtr::create( - match params.aggregate_functions.is_empty() { - true => TransformExchangeGroupBySerializer::create( - self.ctx.clone(), - input, - output, - method.clone(), - operator.clone(), - location_prefix.clone(), - schema.clone(), - local_pos, - compression, - ), - false => TransformExchangeAggregateSerializer::create( - self.ctx.clone(), - input, - output, - method.clone(), - operator.clone(), - location_prefix.clone(), - params.clone(), - compression, - schema.clone(), - local_pos, - ), - }, + TransformExchangeAggregateSerializer::create( + self.ctx.clone(), + input, + output, + operator.clone(), + location_prefix.clone(), + params.clone(), + compression, + schema.clone(), + local_pos, + ), )) })?; @@ -466,18 +320,7 @@ impl ExchangeInjector pipeline: &mut Pipeline, ) -> Result<()> { pipeline.add_transform(|input, output| { - match self.aggregator_params.aggregate_functions.is_empty() { - true => TransformGroupByDeserializer::::try_create( - input, - output, - ¶ms.schema, - ), - false => TransformAggregateDeserializer::::try_create( - input, - output, - ¶ms.schema, - ), - } + TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) }) } @@ -487,18 +330,7 @@ impl ExchangeInjector pipeline: &mut Pipeline, ) -> Result<()> { pipeline.add_transform(|input, output| { - match self.aggregator_params.aggregate_functions.is_empty() { - true => TransformGroupByDeserializer::::try_create( - input, - output, - ¶ms.schema, - ), - false => TransformAggregateDeserializer::::try_create( - input, - output, - ¶ms.schema, - ), - } + TransformAggregateDeserializer::try_create(input, output, ¶ms.schema) }) } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs index a50a348ced60..74b6cf41dbfd 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregate_meta.rs @@ -32,15 +32,6 @@ use databend_common_expression::PartitionedPayload; use databend_common_expression::Payload; use databend_common_expression::ProbeState; -use crate::pipelines::processors::transforms::aggregator::HashTableCell; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; - -pub struct HashTablePayload { - pub bucket: isize, - pub cell: HashTableCell, -} - pub struct SerializedPayload { pub bucket: isize, pub data_block: DataBlock, @@ -123,42 +114,31 @@ pub struct AggregatePayload { pub max_partition_count: usize, } -pub enum AggregateMeta { +pub enum AggregateMeta { Serialized(SerializedPayload), - HashTable(HashTablePayload), AggregatePayload(AggregatePayload), AggregateSpilling(PartitionedPayload), BucketSpilled(BucketSpilledPayload), Spilled(Vec), - Spilling(HashTablePayload, V>), Partitioned { bucket: isize, data: Vec }, } -impl AggregateMeta { - pub fn create_hashtable(bucket: isize, cell: HashTableCell) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::HashTable(HashTablePayload { - cell, - bucket, - })) - } - +impl AggregateMeta { pub fn create_agg_payload( bucket: isize, payload: Payload, max_partition_count: usize, ) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::AggregatePayload( - AggregatePayload { - bucket, - payload, - max_partition_count, - }, - )) + Box::new(AggregateMeta::AggregatePayload(AggregatePayload { + bucket, + payload, + max_partition_count, + })) } pub fn create_agg_spilling(payload: PartitionedPayload) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::AggregateSpilling(payload)) + Box::new(AggregateMeta::AggregateSpilling(payload)) } pub fn create_serialized( @@ -166,64 +146,49 @@ impl AggregateMeta BlockMetaInfoPtr { - Box::new(AggregateMeta::::Serialized(SerializedPayload { + Box::new(AggregateMeta::Serialized(SerializedPayload { bucket, data_block: block, max_partition_count, })) } - pub fn create_spilling( - cell: HashTableCell, V>, - ) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::Spilling(HashTablePayload { - cell, - bucket: 0, - })) - } - pub fn create_spilled(buckets_payload: Vec) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::Spilled(buckets_payload)) + Box::new(AggregateMeta::Spilled(buckets_payload)) } pub fn create_bucket_spilled(payload: BucketSpilledPayload) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::BucketSpilled(payload)) + Box::new(AggregateMeta::BucketSpilled(payload)) } pub fn create_partitioned(bucket: isize, data: Vec) -> BlockMetaInfoPtr { - Box::new(AggregateMeta::::Partitioned { data, bucket }) + Box::new(AggregateMeta::Partitioned { data, bucket }) } } -impl serde::Serialize - for AggregateMeta -{ +impl serde::Serialize for AggregateMeta { fn serialize(&self, _: S) -> std::result::Result where S: serde::Serializer { unreachable!("AggregateMeta does not support exchanging between multiple nodes") } } -impl<'de, Method: HashMethodBounds, V: Send + Sync + 'static> serde::Deserialize<'de> - for AggregateMeta -{ +impl<'de> serde::Deserialize<'de> for AggregateMeta { fn deserialize(_: D) -> std::result::Result where D: serde::Deserializer<'de> { unreachable!("AggregateMeta does not support exchanging between multiple nodes") } } -impl Debug for AggregateMeta { +impl Debug for AggregateMeta { fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { match self { - AggregateMeta::HashTable(_) => f.debug_struct("AggregateMeta::HashTable").finish(), AggregateMeta::Partitioned { .. } => { f.debug_struct("AggregateMeta::Partitioned").finish() } AggregateMeta::Serialized { .. } => { f.debug_struct("AggregateMeta::Serialized").finish() } - AggregateMeta::Spilling(_) => f.debug_struct("Aggregate::Spilling").finish(), AggregateMeta::Spilled(_) => f.debug_struct("Aggregate::Spilled").finish(), AggregateMeta::BucketSpilled(_) => f.debug_struct("Aggregate::BucketSpilled").finish(), AggregateMeta::AggregatePayload(_) => { @@ -236,9 +201,7 @@ impl Debug for AggregateMeta } } -impl BlockMetaInfo - for AggregateMeta -{ +impl BlockMetaInfo for AggregateMeta { fn typetag_deserialize(&self) { unimplemented!("AggregateMeta does not support exchanging between multiple nodes") } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs index f1dfb320c3d0..21571182f244 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/aggregator_params.rs @@ -22,12 +22,9 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_functions::aggregates::get_layout_offsets; use databend_common_functions::aggregates::AggregateFunctionRef; -use databend_common_functions::aggregates::StateAddr; use databend_common_sql::IndexType; use itertools::Itertools; -use crate::pipelines::processors::transforms::group_by::Area; - pub struct AggregatorParams { pub input_schema: DataSchemaRef, pub group_columns: Vec, @@ -81,18 +78,6 @@ impl AggregatorParams { })) } - pub fn alloc_layout(&self, area: &mut Area) -> StateAddr { - let layout = self.layout.unwrap(); - let place = Into::::into(area.alloc_layout(layout)); - - for idx in 0..self.offsets_aggregate_states.len() { - let aggr_state = self.offsets_aggregate_states[idx]; - let aggr_state_place = place.next(aggr_state); - self.aggregate_functions[idx].init_state(aggr_state_place); - } - place - } - pub fn has_distinct_combinator(&self) -> bool { self.aggregate_functions .iter() diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs index 68427aefff55..0c6998e3bcd4 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/mod.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -mod aggregate_cell; mod aggregate_exchange_injector; mod aggregate_meta; mod aggregator_params; @@ -21,23 +20,16 @@ mod serde; mod transform_aggregate_expand; mod transform_aggregate_final; mod transform_aggregate_partial; -mod transform_group_by_final; -mod transform_group_by_partial; -mod transform_partition_bucket; mod transform_single_key; mod utils; -pub use aggregate_cell::HashTableCell; -pub use aggregate_cell::PartitionedHashTableDropper; pub use aggregate_exchange_injector::AggregateInjector; pub use aggregate_meta::*; pub use aggregator_params::AggregatorParams; +pub use new_transform_partition_bucket::build_partition_bucket; pub use transform_aggregate_expand::TransformExpandGroupingSets; pub use transform_aggregate_final::TransformFinalAggregate; pub use transform_aggregate_partial::TransformPartialAggregate; -pub use transform_group_by_final::TransformFinalGroupBy; -pub use transform_group_by_partial::TransformPartialGroupBy; -pub use transform_partition_bucket::build_partition_bucket; pub use transform_single_key::FinalSingleStateAggregator; pub use transform_single_key::PartialSingleStateAggregator; pub use utils::*; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs index 2f4c8cb29036..12c01ecebaef 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/new_transform_partition_bucket.rs @@ -15,7 +15,6 @@ use std::any::Any; use std::collections::btree_map::Entry; use std::collections::BTreeMap; -use std::marker::PhantomData; use std::sync::Arc; use bumpalo::Bump; @@ -29,12 +28,19 @@ use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; +use databend_common_pipeline_core::processors::ProcessorPtr; +use databend_common_pipeline_core::Pipe; +use databend_common_pipeline_core::PipeItem; +use databend_common_pipeline_core::Pipeline; +use databend_common_storage::DataOperator; +use tokio::sync::Semaphore; use super::AggregatePayload; +use super::TransformAggregateSpillReader; +use super::TransformFinalAggregate; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::SerializedPayload; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; static SINGLE_LEVEL_BUCKET_NUM: isize = -1; static MAX_PARTITION_COUNT: usize = 128; @@ -44,7 +50,7 @@ struct InputPortState { bucket: isize, max_partition_count: usize, } -pub struct NewTransformPartitionBucket { +pub struct NewTransformPartitionBucket { output: Arc, inputs: Vec, params: Arc, @@ -56,12 +62,9 @@ pub struct NewTransformPartitionBucket, max_partition_count: usize, - _phantom: PhantomData<(Method, V)>, } -impl - NewTransformPartitionBucket -{ +impl NewTransformPartitionBucket { pub fn create(input_nums: usize, params: Arc) -> Result { let mut inputs = Vec::with_capacity(input_nums); @@ -85,7 +88,6 @@ impl initialized_all_inputs: false, all_inputs_init: false, max_partition_count: 0, - _phantom: Default::default(), }) } @@ -211,17 +213,15 @@ impl fn add_bucket(&mut self, mut data_block: DataBlock) -> Result<(isize, usize)> { let (mut bucket, mut partition_count) = (0, 0); if let Some(block_meta) = data_block.get_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_ref_from(block_meta) { + if let Some(block_meta) = AggregateMeta::downcast_ref_from(block_meta) { (bucket, partition_count) = match block_meta { - AggregateMeta::Spilling(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), AggregateMeta::AggregateSpilling(_) => unreachable!(), - AggregateMeta::HashTable(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => { let meta = data_block.take_meta().unwrap(); if let Some(AggregateMeta::BucketSpilled(payload)) = - AggregateMeta::::downcast_from(meta) + AggregateMeta::downcast_from(meta) { let bucket = payload.bucket; let partition_count = payload.max_partition_count; @@ -229,7 +229,7 @@ impl self.max_partition_count.max(partition_count); let data_block = DataBlock::empty_with_meta( - AggregateMeta::::create_bucket_spilled(payload), + AggregateMeta::create_bucket_spilled(payload), ); match self.buckets_blocks.entry(bucket) { Entry::Vacant(v) => { @@ -248,7 +248,7 @@ impl let meta = data_block.take_meta().unwrap(); if let Some(AggregateMeta::Spilled(buckets_payload)) = - AggregateMeta::::downcast_from(meta) + AggregateMeta::downcast_from(meta) { let partition_count = buckets_payload[0].max_partition_count; self.max_partition_count = @@ -256,12 +256,9 @@ impl for bucket_payload in buckets_payload { let bucket = bucket_payload.bucket; - let data_block = DataBlock::empty_with_meta(AggregateMeta::< - Method, - V, - >::create_bucket_spilled( - bucket_payload - )); + let data_block = DataBlock::empty_with_meta( + AggregateMeta::create_bucket_spilled(bucket_payload), + ); match self.buckets_blocks.entry(bucket) { Entry::Vacant(v) => { v.insert(vec![data_block]); @@ -342,9 +339,8 @@ impl // already is max partition if payload.max_partition_count == self.max_partition_count { let bucket = payload.bucket; - let data_block = DataBlock::empty_with_meta(Box::new( - AggregateMeta::::Serialized(payload), - )); + let data_block = + DataBlock::empty_with_meta(Box::new(AggregateMeta::Serialized(payload))); match self.buckets_blocks.entry(bucket) { Entry::Vacant(v) => { v.insert(vec![data_block]); @@ -375,7 +371,7 @@ impl for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { blocks.push(Some(DataBlock::empty_with_meta( - AggregateMeta::::create_agg_payload( + AggregateMeta::create_agg_payload( bucket as isize, payload, self.max_partition_count, @@ -390,9 +386,8 @@ impl // already is max partition if payload.max_partition_count == self.max_partition_count { let bucket = payload.bucket; - let data_block = DataBlock::empty_with_meta(Box::new( - AggregateMeta::::AggregatePayload(payload), - )); + let data_block = + DataBlock::empty_with_meta(Box::new(AggregateMeta::AggregatePayload(payload))); match self.buckets_blocks.entry(bucket) { Entry::Vacant(v) => { v.insert(vec![data_block]); @@ -417,7 +412,7 @@ impl for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { blocks.push(Some(DataBlock::empty_with_meta( - AggregateMeta::::create_agg_payload( + AggregateMeta::create_agg_payload( bucket as isize, payload, self.max_partition_count, @@ -432,20 +427,18 @@ impl let mut data = Vec::with_capacity(data_blocks.len()); for mut data_block in data_blocks.into_iter() { if let Some(block_meta) = data_block.take_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_from(block_meta) { + if let Some(block_meta) = AggregateMeta::downcast_from(block_meta) { data.push(block_meta); } } } - DataBlock::empty_with_meta(AggregateMeta::::create_partitioned(bucket, data)) + DataBlock::empty_with_meta(AggregateMeta::create_partitioned(bucket, data)) } } #[async_trait::async_trait] -impl Processor - for NewTransformPartitionBucket -{ +impl Processor for NewTransformPartitionBucket { fn name(&self) -> String { String::from("TransformPartitionBucket") } @@ -543,14 +536,12 @@ impl Processor .unpartitioned_blocks .pop() .and_then(|mut block| block.take_meta()) - .and_then(AggregateMeta::::downcast_from); + .and_then(AggregateMeta::downcast_from); if let Some(agg_block_meta) = block_meta { let data_blocks = match agg_block_meta { AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::HashTable(_) => unreachable!(), AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Serialized(payload) => self.partition_block(payload)?, @@ -574,3 +565,38 @@ impl Processor Ok(()) } } + +pub fn build_partition_bucket( + pipeline: &mut Pipeline, + params: Arc, +) -> Result<()> { + let input_nums = pipeline.output_len(); + let transform = NewTransformPartitionBucket::create(input_nums, params.clone())?; + + let output = transform.get_output(); + let inputs_port = transform.get_inputs(); + + pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( + ProcessorPtr::create(Box::new(transform)), + inputs_port, + vec![output], + )])); + + pipeline.try_resize(input_nums)?; + + let semaphore = Arc::new(Semaphore::new(params.max_spill_io_requests)); + let operator = DataOperator::instance().operator(); + pipeline.add_transform(|input, output| { + let operator = operator.clone(); + TransformAggregateSpillReader::create(input, output, operator, semaphore.clone()) + })?; + + pipeline.add_transform(|input, output| { + Ok(ProcessorPtr::create(TransformFinalAggregate::try_create( + input, + output, + params.clone(), + )?)) + })?; + Ok(()) +} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/mod.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/mod.rs index ae0b795b92a3..76a55b10e85b 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/mod.rs @@ -18,9 +18,6 @@ mod transform_aggregate_spill_writer; mod transform_deserializer; mod transform_exchange_aggregate_serializer; mod transform_exchange_async_barrier; -mod transform_exchange_group_by_serializer; -mod transform_group_by_serializer; -mod transform_group_by_spill_writer; mod transform_spill_reader; pub use serde_meta::*; @@ -29,9 +26,6 @@ pub use transform_aggregate_spill_writer::*; pub use transform_deserializer::*; pub use transform_exchange_aggregate_serializer::*; pub use transform_exchange_async_barrier::*; -pub use transform_exchange_group_by_serializer::*; -pub use transform_group_by_serializer::*; -pub use transform_group_by_spill_writer::*; pub use transform_spill_reader::*; pub mod exchange_defines { diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs index 27d721233087..bd3bdb24d633 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_serializer.rs @@ -13,55 +13,47 @@ // limitations under the License. use std::any::Any; +use std::fmt::Formatter; use std::pin::Pin; -use std::ptr::NonNull; use std::sync::Arc; use databend_common_exception::Result; -use databend_common_expression::types::binary::BinaryColumnBuilder; +use databend_common_expression::local_block_meta_serde; +use databend_common_expression::BlockMetaInfo; use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::Column; +use databend_common_expression::BlockMetaInfoPtr; use databend_common_expression::DataBlock; use databend_common_expression::PayloadFlushState; -use databend_common_functions::aggregates::StateAddr; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_core::processors::ProcessorPtr; +use futures::future::BoxFuture; -use super::SerializePayload; -use crate::pipelines::processors::transforms::aggregator::create_state_serializer; -use crate::pipelines::processors::transforms::aggregator::estimated_key_size; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; +use crate::pipelines::processors::transforms::aggregator::AggregatePayload; use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::KeysColumnBuilder; -pub struct TransformAggregateSerializer { - method: Method, +pub struct TransformAggregateSerializer { params: Arc, input: Arc, output: Arc, output_data: Option, - input_data: Option>, + input_data: Option, } -impl TransformAggregateSerializer { +impl TransformAggregateSerializer { pub fn try_create( input: Arc, output: Arc, - method: Method, params: Arc, ) -> Result { Ok(ProcessorPtr::create(Box::new( TransformAggregateSerializer { input, output, - method, params, input_data: None, output_data: None, @@ -70,7 +62,7 @@ impl TransformAggregateSerializer { } } -impl Processor for TransformAggregateSerializer { +impl Processor for TransformAggregateSerializer { fn name(&self) -> String { String::from("TransformAggregateSerializer") } @@ -126,31 +118,21 @@ impl Processor for TransformAggregateSerializer TransformAggregateSerializer { +impl TransformAggregateSerializer { fn transform_input_data(&mut self, mut data_block: DataBlock) -> Result { debug_assert!(data_block.is_empty()); if let Some(block_meta) = data_block.take_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_from(block_meta) { + if let Some(block_meta) = AggregateMeta::downcast_from(block_meta) { match block_meta { AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(_) => unreachable!(), AggregateMeta::Partitioned { .. } => unreachable!(), AggregateMeta::AggregateSpilling(_) => unreachable!(), - AggregateMeta::HashTable(payload) => { - self.input_data = Some(SerializeAggregateStream::create( - &self.method, - &self.params, - SerializePayload::::HashTablePayload(payload), - )); - return Ok(Event::Sync); - } AggregateMeta::AggregatePayload(p) => { self.input_data = Some(SerializeAggregateStream::create( - &self.method, &self.params, - SerializePayload::::AggregatePayload(p), + SerializePayload::AggregatePayload(p), )); return Ok(Event::Sync); } @@ -162,96 +144,65 @@ impl TransformAggregateSerializer { } } -pub fn serialize_aggregate( - method: &Method, - params: &Arc, - hashtable: &Method::HashTable, -) -> Result { - let keys_len = hashtable.len(); - let value_size = estimated_key_size(hashtable); - - let funcs = ¶ms.aggregate_functions; - let offsets_aggregate_states = ¶ms.offsets_aggregate_states; - - // Builders. - let mut state_builders: Vec = funcs - .iter() - .map(|func| create_state_serializer(func, keys_len)) - .collect(); - - let mut group_key_builder = method.keys_column_builder(keys_len, value_size); - - let mut places = Vec::with_capacity(keys_len); - for group_entity in hashtable.iter() { - places.push(Into::::into(*group_entity.get())); - group_key_builder.append_value(group_entity.key()); - } +pub enum SerializePayload { + AggregatePayload(AggregatePayload), +} + +pub enum FlightSerialized { + DataBlock(DataBlock), + Future(BoxFuture<'static, Result>), +} + +unsafe impl Sync for FlightSerialized {} + +pub struct FlightSerializedMeta { + pub serialized_blocks: Vec, +} - let mut columns = Vec::with_capacity(state_builders.len() + 1); - for (idx, func) in funcs.iter().enumerate() { - func.batch_serialize( - &places, - offsets_aggregate_states[idx], - &mut state_builders[idx], - )?; +impl FlightSerializedMeta { + pub fn create(blocks: Vec) -> BlockMetaInfoPtr { + Box::new(FlightSerializedMeta { + serialized_blocks: blocks, + }) } +} - for builder in state_builders.into_iter() { - columns.push(Column::Binary(builder.build())); +impl std::fmt::Debug for FlightSerializedMeta { + fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { + f.debug_struct("FlightSerializedMeta").finish() } - columns.push(group_key_builder.finish()); - Ok(DataBlock::new_from_columns(columns)) } -pub struct SerializeAggregateStream { - method: Method, - params: Arc, - pub payload: Pin>>, - // old hashtable' iter - iter: Option< as HashtableLike>::Iterator<'static>>, - flush_state: Option, +local_block_meta_serde!(FlightSerializedMeta); + +#[typetag::serde(name = "exchange_shuffle")] +impl BlockMetaInfo for FlightSerializedMeta {} + +pub struct SerializeAggregateStream { + _params: Arc, + pub payload: Pin>, + flush_state: PayloadFlushState, end_iter: bool, } -unsafe impl Send for SerializeAggregateStream {} - -unsafe impl Sync for SerializeAggregateStream {} - -impl SerializeAggregateStream { - pub fn create( - method: &Method, - params: &Arc, - payload: SerializePayload, - ) -> Self { - unsafe { - let payload = Box::pin(payload); - - let iter = if let SerializePayload::HashTablePayload(p) = payload.as_ref().get_ref() { - Some(NonNull::from(&p.cell.hashtable).as_ref().iter()) - } else { - None - }; - - let flush_state = - if let SerializePayload::AggregatePayload(_) = payload.as_ref().get_ref() { - Some(PayloadFlushState::default()) - } else { - None - }; - - SerializeAggregateStream:: { - iter, - payload, - end_iter: false, - flush_state, - method: method.clone(), - params: params.clone(), - } +unsafe impl Send for SerializeAggregateStream {} + +unsafe impl Sync for SerializeAggregateStream {} + +impl SerializeAggregateStream { + pub fn create(params: &Arc, payload: SerializePayload) -> Self { + let payload = Box::pin(payload); + + SerializeAggregateStream { + payload, + flush_state: PayloadFlushState::default(), + _params: params.clone(), + end_iter: false, } } } -impl Iterator for SerializeAggregateStream { +impl Iterator for SerializeAggregateStream { type Item = Result; fn next(&mut self) -> Option { @@ -259,58 +210,15 @@ impl Iterator for SerializeAggregateStream { } } -impl SerializeAggregateStream { +impl SerializeAggregateStream { fn next_impl(&mut self) -> Result> { if self.end_iter { return Ok(None); } match self.payload.as_ref().get_ref() { - SerializePayload::HashTablePayload(p) => { - let max_block_rows = std::cmp::min(8192, p.cell.hashtable.len()); - let max_block_bytes = std::cmp::min( - 8 * 1024 * 1024 + 1024, - p.cell.hashtable.unsize_key_size().unwrap_or(usize::MAX), - ); - - let funcs = &self.params.aggregate_functions; - let offsets_aggregate_states = &self.params.offsets_aggregate_states; - - let mut state_builders: Vec = funcs - .iter() - .map(|func| create_state_serializer(func, max_block_rows)) - .collect(); - - let mut group_key_builder = self - .method - .keys_column_builder(max_block_rows, max_block_bytes); - - let mut bytes = 0; - - #[allow(clippy::while_let_on_iterator)] - while let Some(group_entity) = self.iter.as_mut().and_then(|iter| iter.next()) { - let place = Into::::into(*group_entity.get()); - - for (idx, func) in funcs.iter().enumerate() { - let arg_place = place.next(offsets_aggregate_states[idx]); - func.serialize(arg_place, &mut state_builders[idx].data)?; - state_builders[idx].commit_row(); - bytes += state_builders[idx].memory_size(); - } - - group_key_builder.append_value(group_entity.key()); - - if bytes + group_key_builder.bytes_size() >= 8 * 1024 * 1024 { - return self.finish(state_builders, group_key_builder); - } - } - - self.end_iter = true; - self.finish(state_builders, group_key_builder) - } SerializePayload::AggregatePayload(p) => { - let state = self.flush_state.as_mut().unwrap(); - let block = p.payload.aggregate_flush(state)?; + let block = p.payload.aggregate_flush(&mut self.flush_state)?; if block.is_none() { self.end_iter = true; @@ -325,28 +233,4 @@ impl SerializeAggregateStream { } } } - - fn finish( - &self, - state_builders: Vec, - group_key_builder: Method::ColumnBuilder<'_>, - ) -> Result> { - let mut columns = Vec::with_capacity(state_builders.len() + 1); - - for builder in state_builders.into_iter() { - columns.push(Column::Binary(builder.build())); - } - - let bucket = if let SerializePayload::HashTablePayload(p) = self.payload.as_ref().get_ref() - { - p.bucket - } else { - 0 - }; - columns.push(group_key_builder.finish()); - let block = DataBlock::new_from_columns(columns); - Ok(Some( - block.add_meta(Some(AggregateSerdeMeta::create(bucket)))?, - )) - } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs index ab721706d7d2..5bfba9d7f2bb 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_aggregate_spill_writer.rs @@ -27,7 +27,6 @@ use databend_common_expression::arrow::serialize_column; use databend_common_expression::BlockMetaInfoDowncast; use databend_common_expression::DataBlock; use databend_common_expression::PartitionedPayload; -use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::Event; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; @@ -36,45 +35,38 @@ use futures_util::future::BoxFuture; use log::info; use opendal::Operator; -use crate::pipelines::processors::transforms::aggregator::serialize_aggregate; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; -use crate::pipelines::processors::transforms::aggregator::HashTablePayload; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::sessions::QueryContext; -pub struct TransformAggregateSpillWriter { +pub struct TransformAggregateSpillWriter { ctx: Arc, - method: Method, input: Arc, output: Arc, - params: Arc, + _params: Arc, operator: Operator, location_prefix: String, spilled_block: Option, - spilling_meta: Option>, + spilling_meta: Option, spilling_future: Option>>, } -impl TransformAggregateSpillWriter { +impl TransformAggregateSpillWriter { pub fn create( ctx: Arc, input: Arc, output: Arc, - method: Method, operator: Operator, params: Arc, location_prefix: String, ) -> Box { - Box::new(TransformAggregateSpillWriter:: { + Box::new(TransformAggregateSpillWriter { ctx, - method, input, output, - params, + _params: params, operator, location_prefix, spilled_block: None, @@ -85,7 +77,7 @@ impl TransformAggregateSpillWriter { } #[async_trait::async_trait] -impl Processor for TransformAggregateSpillWriter { +impl Processor for TransformAggregateSpillWriter { fn name(&self) -> String { String::from("TransformAggregateSpillWriter") } @@ -127,19 +119,12 @@ impl Processor for TransformAggregateSpillWriter::downcast_ref_from) + .and_then(AggregateMeta::downcast_ref_from) { - if matches!(block_meta, AggregateMeta::Spilling(_)) { - self.input.set_not_need_data(); - let block_meta = data_block.take_meta().unwrap(); - self.spilling_meta = AggregateMeta::::downcast_from(block_meta); - return Ok(Event::Sync); - } - if matches!(block_meta, AggregateMeta::AggregateSpilling(_)) { self.input.set_not_need_data(); let block_meta = data_block.take_meta().unwrap(); - self.spilling_meta = AggregateMeta::::downcast_from(block_meta); + self.spilling_meta = AggregateMeta::downcast_from(block_meta); return Ok(Event::Sync); } } @@ -160,20 +145,8 @@ impl Processor for TransformAggregateSpillWriter Result<()> { if let Some(spilling_meta) = self.spilling_meta.take() { match spilling_meta { - AggregateMeta::Spilling(payload) => { - self.spilling_future = Some(spilling_aggregate_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - &self.params, - payload, - )?); - - return Ok(()); - } AggregateMeta::AggregateSpilling(payload) => { - self.spilling_future = Some(agg_spilling_aggregate_payload::( + self.spilling_future = Some(agg_spilling_aggregate_payload( self.ctx.clone(), self.operator.clone(), &self.location_prefix, @@ -182,7 +155,6 @@ impl Processor for TransformAggregateSpillWriter { return Err(ErrorCode::Internal("")); } @@ -202,7 +174,7 @@ impl Processor for TransformAggregateSpillWriter( +pub fn agg_spilling_aggregate_payload( ctx: Arc, operator: Operator, location_prefix: &str, @@ -294,109 +266,8 @@ pub fn agg_spilling_aggregate_payload( instant.elapsed() ); - Ok(DataBlock::empty_with_meta( - AggregateMeta::::create_spilled(spilled_buckets_payloads), - )) - })) -} - -pub fn spilling_aggregate_payload( - ctx: Arc, - operator: Operator, - method: &Method, - location_prefix: &str, - params: &Arc, - mut payload: HashTablePayload, usize>, -) -> Result>> { - let unique_name = GlobalUniqName::unique(); - let location = format!("{}/{}", location_prefix, unique_name); - - let mut write_size = 0; - let mut write_data = Vec::with_capacity(256); - let mut spilled_buckets_payloads = Vec::with_capacity(256); - // Record how many rows are spilled. - let mut rows = 0; - for (bucket, inner_table) in payload.cell.hashtable.iter_tables_mut().enumerate() { - if inner_table.len() == 0 { - continue; - } - - let data_block = serialize_aggregate(method, params, inner_table)?; - rows += data_block.num_rows(); - - let begin = write_size; - let columns = data_block.columns().to_vec(); - let mut columns_data = Vec::with_capacity(columns.len()); - let mut columns_layout = Vec::with_capacity(columns.len()); - - for column in columns.into_iter() { - let column = column.to_column(data_block.num_rows()); - let column_data = serialize_column(&column); - write_size += column_data.len() as u64; - columns_layout.push(column_data.len() as u64); - columns_data.push(column_data); - } - - write_data.push(columns_data); - spilled_buckets_payloads.push(BucketSpilledPayload { - bucket: bucket as isize, - location: location.clone(), - data_range: begin..write_size, - columns_layout, - max_partition_count: 0, - }); - } - - Ok(Box::pin(async move { - let instant = Instant::now(); - - let mut write_bytes = 0; - - if !write_data.is_empty() { - let mut writer = operator - .writer_with(&location) - .chunk(8 * 1024 * 1024) - .await?; - for write_bucket_data in write_data.into_iter() { - for data in write_bucket_data.into_iter() { - write_bytes += data.len(); - writer.write(data).await?; - } - } - - writer.close().await?; - } - - // perf - { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - write_bytes, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - instant.elapsed().as_millis() as usize, - ); - } - - { - let progress_val = ProgressValues { - rows, - bytes: write_bytes, - }; - ctx.get_aggregate_spill_progress().incr(&progress_val); - } - - info!( - "Write aggregate spill {} successfully, size: {:?} ,elapsed: {:?}", - location, - write_bytes, - instant.elapsed() - ); - - Ok(DataBlock::empty_with_meta( - AggregateMeta::::create_spilled(spilled_buckets_payloads), - )) + Ok(DataBlock::empty_with_meta(AggregateMeta::create_spilled( + spilled_buckets_payloads, + ))) })) } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs index 2f86eef99a47..c73799432d17 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_deserializer.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::marker::PhantomData; use std::sync::Arc; use arrow_schema::Schema as ArrowSchema; @@ -39,19 +38,17 @@ use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; use crate::pipelines::processors::transforms::aggregator::BUCKET_TYPE; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; use crate::servers::flight::v1::exchange::serde::deserialize_block; use crate::servers::flight::v1::exchange::serde::ExchangeDeserializeMeta; use crate::servers::flight::v1::packets::DataPacket; use crate::servers::flight::v1::packets::FragmentData; -pub struct TransformDeserializer { +pub struct TransformDeserializer { schema: DataSchemaRef, arrow_schema: Arc, - _phantom: PhantomData<(Method, V)>, } -impl TransformDeserializer { +impl TransformDeserializer { pub fn try_create( input: Arc, output: Arc, @@ -62,10 +59,9 @@ impl TransformDeserializer { + TransformDeserializer { arrow_schema: Arc::new(arrow_schema), schema: schema.clone(), - _phantom: Default::default(), }, ))) } @@ -94,7 +90,7 @@ impl TransformDeserializer { return match meta.typ == BUCKET_TYPE { true => Ok(DataBlock::empty_with_meta( - AggregateMeta::::create_serialized( + AggregateMeta::create_serialized( meta.bucket, deserialize_block( dict, @@ -151,9 +147,9 @@ impl TransformDeserializer::create_spilled(buckets_payload), - )) + Ok(DataBlock::empty_with_meta(AggregateMeta::create_spilled( + buckets_payload, + ))) } }; } @@ -167,11 +163,7 @@ impl TransformDeserializer BlockMetaTransform for TransformDeserializer -where - M: HashMethodBounds, - V: Send + Sync + 'static, -{ +impl BlockMetaTransform for TransformDeserializer { const UNKNOWN_MODE: UnknownMode = UnknownMode::Pass; const NAME: &'static str = "TransformDeserializer"; @@ -189,5 +181,4 @@ where } } -pub type TransformGroupByDeserializer = TransformDeserializer; -pub type TransformAggregateDeserializer = TransformDeserializer; +pub type TransformAggregateDeserializer = TransformDeserializer; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_aggregate_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_aggregate_serializer.rs index b2b65acbce7a..64422ad91687 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_aggregate_serializer.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_aggregate_serializer.rs @@ -34,7 +34,6 @@ use databend_common_expression::DataBlock; use databend_common_expression::DataSchemaRef; use databend_common_expression::FromData; use databend_common_expression::PartitionedPayload; -use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; @@ -49,24 +48,18 @@ use super::SerializePayload; use crate::pipelines::processors::transforms::aggregator::agg_spilling_aggregate_payload as local_agg_spilling_aggregate_payload; use crate::pipelines::processors::transforms::aggregator::aggregate_exchange_injector::compute_block_number; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::HashTablePayload; use crate::pipelines::processors::transforms::aggregator::exchange_defines; -use crate::pipelines::processors::transforms::aggregator::serialize_aggregate; -use crate::pipelines::processors::transforms::aggregator::spilling_aggregate_payload as local_spilling_aggregate_payload; use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; use crate::pipelines::processors::transforms::aggregator::FlightSerialized; use crate::pipelines::processors::transforms::aggregator::FlightSerializedMeta; use crate::pipelines::processors::transforms::aggregator::SerializeAggregateStream; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; use crate::servers::flight::v1::exchange::serde::serialize_block; use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; use crate::sessions::QueryContext; -pub struct TransformExchangeAggregateSerializer { +pub struct TransformExchangeAggregateSerializer { ctx: Arc, - method: Method, local_pos: usize, options: IpcWriteOptions, @@ -75,13 +68,13 @@ pub struct TransformExchangeAggregateSerializer { params: Arc, } -impl TransformExchangeAggregateSerializer { +impl TransformExchangeAggregateSerializer { #[allow(clippy::too_many_arguments)] pub fn create( ctx: Arc, input: Arc, output: Arc, - method: Method, + operator: Operator, location_prefix: String, params: Arc, @@ -97,11 +90,8 @@ impl TransformExchangeAggregateSerializer { }, }; - BlockMetaTransformer::create(input, output, TransformExchangeAggregateSerializer::< - Method, - > { + BlockMetaTransformer::create(input, output, TransformExchangeAggregateSerializer { ctx, - method, params, operator, location_prefix, @@ -113,9 +103,7 @@ impl TransformExchangeAggregateSerializer { } } -impl BlockMetaTransform - for TransformExchangeAggregateSerializer -{ +impl BlockMetaTransform for TransformExchangeAggregateSerializer { const NAME: &'static str = "TransformExchangeAggregateSerializer"; fn transform(&mut self, meta: ExchangeShuffleMeta) -> Result> { @@ -126,38 +114,16 @@ impl BlockMetaTransform continue; } - match AggregateMeta::::downcast_from(block.take_meta().unwrap()) { + match AggregateMeta::downcast_from(block.take_meta().unwrap()) { None => unreachable!(), Some(AggregateMeta::Spilled(_)) => unreachable!(), Some(AggregateMeta::Serialized(_)) => unreachable!(), Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), Some(AggregateMeta::Partitioned { .. }) => unreachable!(), - Some(AggregateMeta::Spilling(payload)) => { - serialized_blocks.push(FlightSerialized::Future( - match index == self.local_pos { - true => local_spilling_aggregate_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - &self.params, - payload, - )?, - false => spilling_aggregate_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - &self.params, - payload, - )?, - }, - )); - } Some(AggregateMeta::AggregateSpilling(payload)) => { serialized_blocks.push(FlightSerialized::Future( match index == self.local_pos { - true => local_agg_spilling_aggregate_payload::( + true => local_agg_spilling_aggregate_payload( self.ctx.clone(), self.operator.clone(), &self.location_prefix, @@ -172,49 +138,19 @@ impl BlockMetaTransform }, )); } - Some(AggregateMeta::HashTable(payload)) => { - if index == self.local_pos { - serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( - Some(Box::new(AggregateMeta::::HashTable(payload))), - )?)); - continue; - } - let bucket = payload.bucket; - let stream = SerializeAggregateStream::create( - &self.method, - &self.params, - SerializePayload::::HashTablePayload(payload), - ); - let mut stream_blocks = stream.into_iter().collect::>>()?; - - if stream_blocks.is_empty() { - serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); - } else { - let mut c = DataBlock::concat(&stream_blocks)?; - if let Some(meta) = stream_blocks[0].take_meta() { - c.replace_meta(meta); - } - - let c = serialize_block(bucket, c, &self.options)?; - serialized_blocks.push(FlightSerialized::DataBlock(c)); - } - } Some(AggregateMeta::AggregatePayload(p)) => { if index == self.local_pos { - serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( - Some(Box::new(AggregateMeta::::AggregatePayload( - p, - ))), - )?)); + serialized_blocks.push(FlightSerialized::DataBlock( + block.add_meta(Some(Box::new(AggregateMeta::AggregatePayload(p))))?, + )); continue; } let bucket = compute_block_number(p.bucket, p.max_partition_count)?; let stream = SerializeAggregateStream::create( - &self.method, &self.params, - SerializePayload::::AggregatePayload(p), + SerializePayload::AggregatePayload(p), ); let mut stream_blocks = stream.into_iter().collect::>>()?; @@ -360,125 +296,3 @@ fn agg_spilling_aggregate_payload( Ok(DataBlock::empty()) })) } - -fn spilling_aggregate_payload( - ctx: Arc, - operator: Operator, - method: &Method, - location_prefix: &str, - params: &Arc, - mut payload: HashTablePayload, usize>, -) -> Result>> { - let unique_name = GlobalUniqName::unique(); - let location = format!("{}/{}", location_prefix, unique_name); - - let mut write_size = 0; - let mut write_data = Vec::with_capacity(256); - let mut buckets_column_data = Vec::with_capacity(256); - let mut data_range_start_column_data = Vec::with_capacity(256); - let mut data_range_end_column_data = Vec::with_capacity(256); - let mut columns_layout_column_data = Vec::with_capacity(256); - // Record how many rows are spilled. - let mut rows = 0; - - for (bucket, inner_table) in payload.cell.hashtable.iter_tables_mut().enumerate() { - if inner_table.len() == 0 { - continue; - } - - let data_block = serialize_aggregate(method, params, inner_table)?; - rows += data_block.num_rows(); - - let old_write_size = write_size; - let columns = data_block.columns().to_vec(); - let mut columns_data = Vec::with_capacity(columns.len()); - let mut columns_layout = Vec::with_capacity(columns.len()); - - for column in columns.into_iter() { - let column = column.to_column(data_block.num_rows()); - let column_data = serialize_column(&column); - write_size += column_data.len() as u64; - columns_layout.push(column_data.len() as u64); - columns_data.push(column_data); - } - - write_data.push(columns_data); - buckets_column_data.push(bucket as i64); - data_range_end_column_data.push(write_size); - columns_layout_column_data.push(columns_layout); - data_range_start_column_data.push(old_write_size); - } - - Ok(Box::pin(async move { - if !write_data.is_empty() { - let instant = Instant::now(); - - let mut write_bytes = 0; - let mut writer = operator - .writer_with(&location) - .chunk(8 * 1024 * 1024) - .await?; - for write_bucket_data in write_data.into_iter() { - for data in write_bucket_data.into_iter() { - write_bytes += data.len(); - writer.write(data).await?; - } - } - - writer.close().await?; - - // perf - { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - write_bytes, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - instant.elapsed().as_millis() as usize, - ); - } - - { - { - let progress_val = ProgressValues { - rows, - bytes: write_bytes, - }; - ctx.get_aggregate_spill_progress().incr(&progress_val); - } - } - - info!( - "Write aggregate spill {} successfully, elapsed: {:?}", - location, - instant.elapsed() - ); - - let data_block = DataBlock::new_from_columns(vec![ - Int64Type::from_data(buckets_column_data), - UInt64Type::from_data(data_range_start_column_data), - UInt64Type::from_data(data_range_end_column_data), - ArrayType::upcast_column(ArrayType::::column_from_iter( - columns_layout_column_data - .into_iter() - .map(|x| UInt64Type::column_from_iter(x.into_iter(), &[])), - &[], - )), - ]); - - let data_block = data_block.add_meta(Some(AggregateSerdeMeta::create_spilled( - -1, - location.clone(), - 0..0, - vec![], - )))?; - - let write_options = exchange_defines::spilled_write_options(); - return serialize_block(-1, data_block, &write_options); - } - - Ok(DataBlock::empty()) - })) -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs deleted file mode 100644 index 33dbcba34360..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_exchange_group_by_serializer.rs +++ /dev/null @@ -1,515 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::fmt::Debug; -use std::fmt::Formatter; -use std::sync::Arc; -use std::time::Instant; - -use arrow_ipc::writer::IpcWriteOptions; -use arrow_ipc::CompressionType; -use databend_common_base::base::GlobalUniqName; -use databend_common_base::base::ProgressValues; -use databend_common_base::runtime::profile::Profile; -use databend_common_base::runtime::profile::ProfileStatisticsName; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::Result; -use databend_common_expression::arrow::serialize_column; -use databend_common_expression::local_block_meta_serde; -use databend_common_expression::types::ArgType; -use databend_common_expression::types::ArrayType; -use databend_common_expression::types::Int64Type; -use databend_common_expression::types::UInt64Type; -use databend_common_expression::types::ValueType; -use databend_common_expression::BlockEntry; -use databend_common_expression::BlockMetaInfo; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::BlockMetaInfoPtr; -use databend_common_expression::DataBlock; -use databend_common_expression::DataSchemaRef; -use databend_common_expression::FromData; -use databend_common_expression::PartitionedPayload; -use databend_common_hashtable::HashtableLike; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::Processor; -use databend_common_pipeline_transforms::processors::BlockMetaTransform; -use databend_common_pipeline_transforms::processors::BlockMetaTransformer; -use databend_common_pipeline_transforms::processors::UnknownMode; -use databend_common_settings::FlightCompression; -use futures_util::future::BoxFuture; -use log::info; -use opendal::Operator; - -use super::SerializePayload; -use crate::pipelines::processors::transforms::aggregator::agg_spilling_group_by_payload as local_agg_spilling_group_by_payload; -use crate::pipelines::processors::transforms::aggregator::aggregate_exchange_injector::compute_block_number; -use crate::pipelines::processors::transforms::aggregator::exchange_defines; -use crate::pipelines::processors::transforms::aggregator::serialize_group_by; -use crate::pipelines::processors::transforms::aggregator::spilling_group_by_payload as local_spilling_group_by_payload; -use crate::pipelines::processors::transforms::aggregator::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; -use crate::pipelines::processors::transforms::aggregator::HashTablePayload; -use crate::pipelines::processors::transforms::aggregator::SerializeGroupByStream; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; -use crate::servers::flight::v1::exchange::serde::serialize_block; -use crate::servers::flight::v1::exchange::ExchangeShuffleMeta; -use crate::sessions::QueryContext; - -pub struct TransformExchangeGroupBySerializer { - ctx: Arc, - method: Method, - local_pos: usize, - options: IpcWriteOptions, - - operator: Operator, - location_prefix: String, -} - -impl TransformExchangeGroupBySerializer { - #[allow(clippy::too_many_arguments)] - pub fn create( - ctx: Arc, - input: Arc, - output: Arc, - method: Method, - operator: Operator, - location_prefix: String, - _schema: DataSchemaRef, - local_pos: usize, - compression: Option, - ) -> Box { - let compression = match compression { - None => None, - Some(compression) => match compression { - FlightCompression::Lz4 => Some(CompressionType::LZ4_FRAME), - FlightCompression::Zstd => Some(CompressionType::ZSTD), - }, - }; - - BlockMetaTransformer::create( - input, - output, - TransformExchangeGroupBySerializer:: { - ctx, - method, - operator, - local_pos, - location_prefix, - options: IpcWriteOptions::default() - .try_with_compression(compression) - .unwrap(), - }, - ) - } -} - -pub enum FlightSerialized { - DataBlock(DataBlock), - Future(BoxFuture<'static, Result>), -} - -unsafe impl Sync for FlightSerialized {} - -pub struct FlightSerializedMeta { - pub serialized_blocks: Vec, -} - -impl FlightSerializedMeta { - pub fn create(blocks: Vec) -> BlockMetaInfoPtr { - Box::new(FlightSerializedMeta { - serialized_blocks: blocks, - }) - } -} - -impl Debug for FlightSerializedMeta { - fn fmt(&self, f: &mut Formatter) -> std::fmt::Result { - f.debug_struct("FlightSerializedMeta").finish() - } -} - -local_block_meta_serde!(FlightSerializedMeta); - -#[typetag::serde(name = "exchange_shuffle")] -impl BlockMetaInfo for FlightSerializedMeta {} - -impl BlockMetaTransform - for TransformExchangeGroupBySerializer -{ - const UNKNOWN_MODE: UnknownMode = UnknownMode::Error; - const NAME: &'static str = "TransformExchangeGroupBySerializer"; - - fn transform(&mut self, meta: ExchangeShuffleMeta) -> Result> { - let mut serialized_blocks = Vec::with_capacity(meta.blocks.len()); - for (index, mut block) in meta.blocks.into_iter().enumerate() { - if block.is_empty() && block.get_meta().is_none() { - serialized_blocks.push(FlightSerialized::DataBlock(block)); - continue; - } - - match AggregateMeta::::downcast_from(block.take_meta().unwrap()) { - None => unreachable!(), - Some(AggregateMeta::Spilled(_)) => unreachable!(), - Some(AggregateMeta::BucketSpilled(_)) => unreachable!(), - Some(AggregateMeta::Serialized(_)) => unreachable!(), - Some(AggregateMeta::Partitioned { .. }) => unreachable!(), - Some(AggregateMeta::Spilling(payload)) => { - serialized_blocks.push(FlightSerialized::Future( - match index == self.local_pos { - true => local_spilling_group_by_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - payload, - )?, - false => spilling_group_by_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - payload, - )?, - }, - )); - } - Some(AggregateMeta::AggregateSpilling(payload)) => { - serialized_blocks.push(FlightSerialized::Future( - match index == self.local_pos { - true => local_agg_spilling_group_by_payload::( - self.ctx.clone(), - self.operator.clone(), - &self.location_prefix, - payload, - )?, - false => agg_spilling_group_by_payload( - self.ctx.clone(), - self.operator.clone(), - &self.location_prefix, - payload, - )?, - }, - )); - } - Some(AggregateMeta::HashTable(payload)) => { - if index == self.local_pos { - serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( - Some(Box::new(AggregateMeta::::HashTable(payload))), - )?)); - continue; - } - - let bucket = payload.bucket; - let stream = SerializeGroupByStream::create( - &self.method, - SerializePayload::::HashTablePayload(payload), - ); - - let mut stream_blocks = stream.into_iter().collect::>>()?; - - if stream_blocks.is_empty() { - serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); - } else { - let mut c = DataBlock::concat(&stream_blocks)?; - if let Some(meta) = stream_blocks[0].take_meta() { - c.replace_meta(meta); - } - - let c = serialize_block(bucket, c, &self.options)?; - serialized_blocks.push(FlightSerialized::DataBlock(c)); - } - } - Some(AggregateMeta::AggregatePayload(p)) => { - if index == self.local_pos { - serialized_blocks.push(FlightSerialized::DataBlock(block.add_meta( - Some(Box::new(AggregateMeta::::AggregatePayload(p))), - )?)); - continue; - } - - let bucket = compute_block_number(p.bucket, p.max_partition_count)?; - let stream = SerializeGroupByStream::create( - &self.method, - SerializePayload::::AggregatePayload(p), - ); - - let mut stream_blocks = stream.into_iter().collect::>>()?; - - if stream_blocks.is_empty() { - serialized_blocks.push(FlightSerialized::DataBlock(DataBlock::empty())); - } else { - let mut c = DataBlock::concat(&stream_blocks)?; - if let Some(meta) = stream_blocks[0].take_meta() { - c.replace_meta(meta); - } - - let c = serialize_block(bucket, c, &self.options)?; - serialized_blocks.push(FlightSerialized::DataBlock(c)); - } - } - }; - } - - Ok(vec![DataBlock::empty_with_meta( - FlightSerializedMeta::create(serialized_blocks), - )]) - } -} - -fn get_columns(data_block: DataBlock) -> Vec { - data_block.columns().to_vec() -} - -fn agg_spilling_group_by_payload( - ctx: Arc, - operator: Operator, - location_prefix: &str, - partitioned_payload: PartitionedPayload, -) -> Result>> { - let unique_name = GlobalUniqName::unique(); - let location = format!("{}/{}", location_prefix, unique_name); - - let mut write_size = 0; - let partition_count = partitioned_payload.partition_count(); - let mut write_data = Vec::with_capacity(partition_count); - let mut buckets_column_data = Vec::with_capacity(partition_count); - let mut data_range_start_column_data = Vec::with_capacity(partition_count); - let mut data_range_end_column_data = Vec::with_capacity(partition_count); - let mut columns_layout_column_data = Vec::with_capacity(partition_count); - // Record how many rows are spilled - let mut rows = 0; - - for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { - if payload.len() == 0 { - continue; - } - - let data_block = payload.group_by_flush_all()?; - let num_rows = data_block.num_rows(); - rows += data_block.num_rows(); - - let old_write_size = write_size; - let columns = get_columns(data_block); - let mut columns_data = Vec::with_capacity(columns.len()); - let mut columns_layout = Vec::with_capacity(columns.len()); - - for column in columns.into_iter() { - let column = column.to_column(num_rows); - let column_data = serialize_column(&column); - write_size += column_data.len() as u64; - columns_layout.push(column_data.len() as u64); - columns_data.push(column_data); - } - - write_data.push(columns_data); - buckets_column_data.push(bucket as i64); - data_range_end_column_data.push(write_size); - columns_layout_column_data.push(columns_layout); - data_range_start_column_data.push(old_write_size); - } - - Ok(Box::pin(async move { - let instant = Instant::now(); - - if !write_data.is_empty() { - let mut write_bytes = 0; - let mut writer = operator - .writer_with(&location) - .chunk(8 * 1024 * 1024) - .await?; - for write_bucket_data in write_data.into_iter() { - for data in write_bucket_data.into_iter() { - write_bytes += data.len(); - writer.write(data).await?; - } - } - - writer.close().await?; - - // perf - { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - write_bytes, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - instant.elapsed().as_millis() as usize, - ); - } - - { - let progress_val = ProgressValues { - rows, - bytes: write_bytes, - }; - ctx.get_group_by_spill_progress().incr(&progress_val); - } - - info!( - "Write aggregate spill {} successfully, elapsed: {:?}", - location, - instant.elapsed() - ); - - let data_block = DataBlock::new_from_columns(vec![ - Int64Type::from_data(buckets_column_data), - UInt64Type::from_data(data_range_start_column_data), - UInt64Type::from_data(data_range_end_column_data), - ArrayType::upcast_column(ArrayType::::column_from_iter( - columns_layout_column_data - .into_iter() - .map(|x| UInt64Type::column_from_iter(x.into_iter(), &[])), - &[], - )), - ]); - - let data_block = data_block.add_meta(Some(AggregateSerdeMeta::create_agg_spilled( - -1, - location.clone(), - 0..0, - vec![], - partition_count, - )))?; - - let write_options = exchange_defines::spilled_write_options(); - return serialize_block(-1, data_block, &write_options); - } - - Ok(DataBlock::empty()) - })) -} - -fn spilling_group_by_payload( - ctx: Arc, - operator: Operator, - method: &Method, - location_prefix: &str, - mut payload: HashTablePayload, ()>, -) -> Result>> { - let unique_name = GlobalUniqName::unique(); - let location = format!("{}/{}", location_prefix, unique_name); - - let mut write_size = 0; - let mut write_data = Vec::with_capacity(256); - let mut buckets_column_data = Vec::with_capacity(256); - let mut data_range_start_column_data = Vec::with_capacity(256); - let mut data_range_end_column_data = Vec::with_capacity(256); - let mut columns_layout_column_data = Vec::with_capacity(256); - // Record how many rows are spilled - let mut rows = 0; - - for (bucket, inner_table) in payload.cell.hashtable.iter_tables_mut().enumerate() { - if inner_table.len() == 0 { - continue; - } - - let data_block = serialize_group_by(method, inner_table)?; - let num_rows = data_block.num_rows(); - rows += 0; - - let old_write_size = write_size; - let columns = get_columns(data_block); - let mut columns_data = Vec::with_capacity(columns.len()); - let mut columns_layout = Vec::with_capacity(columns.len()); - - for column in columns.into_iter() { - let column = column.to_column(num_rows); - let column_data = serialize_column(&column); - write_size += column_data.len() as u64; - columns_layout.push(column_data.len() as u64); - columns_data.push(column_data); - } - - write_data.push(columns_data); - buckets_column_data.push(bucket as i64); - data_range_end_column_data.push(write_size); - columns_layout_column_data.push(columns_layout); - data_range_start_column_data.push(old_write_size); - } - - Ok(Box::pin(async move { - let instant = Instant::now(); - - if !write_data.is_empty() { - let mut write_bytes = 0; - let mut writer = operator - .writer_with(&location) - .chunk(8 * 1024 * 1024) - .await?; - for write_bucket_data in write_data.into_iter() { - for data in write_bucket_data.into_iter() { - write_bytes += data.len(); - writer.write(data).await?; - } - } - - writer.close().await?; - - // perf - { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - write_bytes, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - instant.elapsed().as_millis() as usize, - ); - } - - { - let progress_val = ProgressValues { - rows, - bytes: write_bytes, - }; - ctx.get_group_by_spill_progress().incr(&progress_val); - } - - info!( - "Write aggregate spill {} successfully, elapsed: {:?}", - location, - instant.elapsed() - ); - - let data_block = DataBlock::new_from_columns(vec![ - Int64Type::from_data(buckets_column_data), - UInt64Type::from_data(data_range_start_column_data), - UInt64Type::from_data(data_range_end_column_data), - ArrayType::upcast_column(ArrayType::::column_from_iter( - columns_layout_column_data - .into_iter() - .map(|x| UInt64Type::column_from_iter(x.into_iter(), &[])), - &[], - )), - ]); - - let data_block = data_block.add_meta(Some(AggregateSerdeMeta::create_spilled( - -1, - location.clone(), - 0..0, - vec![], - )))?; - - let write_options = exchange_defines::spilled_write_options(); - return serialize_block(-1, data_block, &write_options); - } - - Ok(DataBlock::empty()) - })) -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs deleted file mode 100644 index a0d70a88aa54..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_serializer.rs +++ /dev/null @@ -1,272 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::any::Any; -use std::pin::Pin; -use std::ptr::NonNull; -use std::sync::Arc; - -use databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_expression::PayloadFlushState; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; -use databend_common_pipeline_core::processors::Event; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::Processor; -use databend_common_pipeline_core::processors::ProcessorPtr; - -use crate::pipelines::processors::transforms::aggregator::estimated_key_size; -use crate::pipelines::processors::transforms::aggregator::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::AggregatePayload; -use crate::pipelines::processors::transforms::aggregator::AggregateSerdeMeta; -use crate::pipelines::processors::transforms::aggregator::HashTablePayload; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::KeysColumnBuilder; - -pub struct TransformGroupBySerializer { - method: Method, - input: Arc, - output: Arc, - output_data: Option, - input_data: Option>, -} - -impl TransformGroupBySerializer { - pub fn try_create( - input: Arc, - output: Arc, - method: Method, - ) -> Result { - Ok(ProcessorPtr::create(Box::new(TransformGroupBySerializer { - method, - input, - output, - output_data: None, - input_data: None, - }))) - } -} - -impl Processor for TransformGroupBySerializer { - fn name(&self) -> String { - String::from("TransformGroupBySerializer") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if self.output.is_finished() { - self.input.finish(); - return Ok(Event::Finished); - } - - if !self.output.can_push() { - self.input.set_not_need_data(); - return Ok(Event::NeedConsume); - } - - if let Some(output_data) = self.output_data.take() { - self.output.push_data(Ok(output_data)); - return Ok(Event::NeedConsume); - } - - if self.input_data.is_some() { - return Ok(Event::Sync); - } - - if self.input.has_data() { - let data_block = self.input.pull_data().unwrap()?; - return self.transform_input_data(data_block); - } - - if self.input.is_finished() { - self.output.finish(); - return Ok(Event::Finished); - } - - self.input.set_need_data(); - Ok(Event::NeedData) - } - - fn process(&mut self) -> Result<()> { - if let Some(stream) = &mut self.input_data { - self.output_data = Option::transpose(stream.next())?; - - if self.output_data.is_none() { - self.input_data = None; - } - } - - Ok(()) - } -} - -impl TransformGroupBySerializer { - fn transform_input_data(&mut self, mut data_block: DataBlock) -> Result { - debug_assert!(data_block.is_empty()); - if let Some(block_meta) = data_block.take_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_from(block_meta) { - match block_meta { - AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), - AggregateMeta::Serialized(_) => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), - AggregateMeta::AggregatePayload(p) => { - self.input_data = Some(SerializeGroupByStream::create( - &self.method, - SerializePayload::::AggregatePayload(p), - )); - return Ok(Event::Sync); - } - AggregateMeta::HashTable(payload) => { - self.input_data = Some(SerializeGroupByStream::create( - &self.method, - SerializePayload::::HashTablePayload(payload), - )); - return Ok(Event::Sync); - } - } - } - } - - unreachable!() - } -} - -pub fn serialize_group_by( - method: &Method, - hashtable: &Method::HashTable<()>, -) -> Result { - let keys_len = hashtable.len(); - let value_size = estimated_key_size(hashtable); - let mut group_key_builder = method.keys_column_builder(keys_len, value_size); - - for group_entity in hashtable.iter() { - group_key_builder.append_value(group_entity.key()); - } - - Ok(DataBlock::new_from_columns( - vec![group_key_builder.finish()], - )) -} - -pub enum SerializePayload { - HashTablePayload(HashTablePayload), - AggregatePayload(AggregatePayload), -} - -pub struct SerializeGroupByStream { - method: Method, - pub payload: Pin>>, - // old hashtable' iter - iter: Option< as HashtableLike>::Iterator<'static>>, - flush_state: Option, - end_iter: bool, -} - -unsafe impl Send for SerializeGroupByStream {} - -unsafe impl Sync for SerializeGroupByStream {} - -impl SerializeGroupByStream { - pub fn create(method: &Method, payload: SerializePayload) -> Self { - unsafe { - let payload = Box::pin(payload); - - let iter = if let SerializePayload::HashTablePayload(p) = payload.as_ref().get_ref() { - Some(NonNull::from(&p.cell.hashtable).as_ref().iter()) - } else { - None - }; - - let flush_state = - if let SerializePayload::AggregatePayload(_) = payload.as_ref().get_ref() { - Some(PayloadFlushState::default()) - } else { - None - }; - - SerializeGroupByStream:: { - iter, - payload, - flush_state, - method: method.clone(), - end_iter: false, - } - } - } -} - -impl Iterator for SerializeGroupByStream { - type Item = Result; - - fn next(&mut self) -> Option { - if self.end_iter { - return None; - } - - match self.payload.as_ref().get_ref() { - SerializePayload::HashTablePayload(p) => { - let max_block_rows = std::cmp::min(8192, p.cell.hashtable.len()); - let max_block_bytes = std::cmp::min( - 8 * 1024 * 1024 + 1024, - p.cell.hashtable.unsize_key_size().unwrap_or(usize::MAX), - ); - - let mut group_key_builder = self - .method - .keys_column_builder(max_block_rows, max_block_bytes); - - #[allow(clippy::while_let_on_iterator)] - while let Some(group_entity) = self.iter.as_mut()?.next() { - group_key_builder.append_value(group_entity.key()); - - if group_key_builder.bytes_size() >= 8 * 1024 * 1024 { - let bucket = p.bucket; - let data_block = - DataBlock::new_from_columns(vec![group_key_builder.finish()]); - return Some(data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))); - } - } - - self.end_iter = true; - let bucket = p.bucket; - let data_block = DataBlock::new_from_columns(vec![group_key_builder.finish()]); - Some(data_block.add_meta(Some(AggregateSerdeMeta::create(bucket)))) - } - SerializePayload::AggregatePayload(p) => { - let state = self.flush_state.as_mut().unwrap(); - let block = p.payload.aggregate_flush(state).unwrap(); - - if block.is_none() { - self.end_iter = true; - } - - block.map(|block| { - block.add_meta(Some(AggregateSerdeMeta::create_agg_payload( - p.bucket, - p.max_partition_count, - ))) - }) - } - } - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs deleted file mode 100644 index 04d0c36b7f3d..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_group_by_spill_writer.rs +++ /dev/null @@ -1,395 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::any::Any; -use std::sync::Arc; -use std::time::Instant; - -use databend_common_base::base::GlobalUniqName; -use databend_common_base::base::ProgressValues; -use databend_common_base::runtime::profile::Profile; -use databend_common_base::runtime::profile::ProfileStatisticsName; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::arrow::serialize_column; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_expression::PartitionedPayload; -use databend_common_hashtable::HashtableLike; -use databend_common_pipeline_core::processors::Event; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::Processor; -use futures_util::future::BoxFuture; -use log::info; -use opendal::Operator; - -use crate::pipelines::processors::transforms::aggregator::serialize_group_by; -use crate::pipelines::processors::transforms::aggregator::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; -use crate::pipelines::processors::transforms::aggregator::HashTablePayload; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; -use crate::sessions::QueryContext; - -pub struct TransformGroupBySpillWriter { - ctx: Arc, - method: Method, - input: Arc, - output: Arc, - - operator: Operator, - location_prefix: String, - spilled_block: Option, - spilling_meta: Option>, - spilling_future: Option>>, -} - -impl TransformGroupBySpillWriter { - pub fn create( - ctx: Arc, - input: Arc, - output: Arc, - method: Method, - operator: Operator, - location_prefix: String, - ) -> Box { - Box::new(TransformGroupBySpillWriter:: { - ctx, - method, - input, - output, - operator, - location_prefix, - spilled_block: None, - spilling_meta: None, - spilling_future: None, - }) - } -} - -#[async_trait::async_trait] -impl Processor for TransformGroupBySpillWriter { - fn name(&self) -> String { - String::from("TransformGroupBySpillWriter") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if self.output.is_finished() { - self.input.finish(); - return Ok(Event::Finished); - } - - if !self.output.can_push() { - self.input.set_not_need_data(); - return Ok(Event::NeedConsume); - } - - if self.spilling_future.is_some() { - self.input.set_not_need_data(); - return Ok(Event::Async); - } - - if let Some(spilled_block) = self.spilled_block.take() { - if !spilled_block.is_empty() || spilled_block.get_meta().is_some() { - self.output.push_data(Ok(spilled_block)); - return Ok(Event::NeedConsume); - } - } - - if self.spilling_meta.is_some() { - self.input.set_not_need_data(); - return Ok(Event::Sync); - } - - if self.input.has_data() { - let mut data_block = self.input.pull_data().unwrap()?; - - if let Some(block_meta) = data_block - .get_meta() - .and_then(AggregateMeta::::downcast_ref_from) - { - if matches!(block_meta, AggregateMeta::Spilling(_)) { - self.input.set_not_need_data(); - let block_meta = data_block.take_meta().unwrap(); - self.spilling_meta = AggregateMeta::::downcast_from(block_meta); - return Ok(Event::Sync); - } - - if matches!(block_meta, AggregateMeta::AggregateSpilling(_)) { - self.input.set_not_need_data(); - let block_meta = data_block.take_meta().unwrap(); - self.spilling_meta = AggregateMeta::::downcast_from(block_meta); - return Ok(Event::Sync); - } - } - - self.output.push_data(Ok(data_block)); - return Ok(Event::NeedConsume); - } - - if self.input.is_finished() { - self.output.finish(); - return Ok(Event::Finished); - } - - self.input.set_need_data(); - Ok(Event::NeedData) - } - - fn process(&mut self) -> Result<()> { - if let Some(spilling_meta) = self.spilling_meta.take() { - match spilling_meta { - AggregateMeta::Spilling(payload) => { - self.spilling_future = Some(spilling_group_by_payload( - self.ctx.clone(), - self.operator.clone(), - &self.method, - &self.location_prefix, - payload, - )?); - - return Ok(()); - } - AggregateMeta::AggregateSpilling(payload) => { - self.spilling_future = Some(agg_spilling_group_by_payload::( - self.ctx.clone(), - self.operator.clone(), - &self.location_prefix, - payload, - )?); - - return Ok(()); - } - - _ => { - return Err(ErrorCode::Internal( - "TransformGroupBySpillWriter only recv AggregateMeta", - )); - } - } - } - - Ok(()) - } - - #[async_backtrace::framed] - async fn async_process(&mut self) -> Result<()> { - if let Some(spilling_future) = self.spilling_future.take() { - self.spilled_block = Some(spilling_future.await?); - } - - Ok(()) - } -} - -pub fn agg_spilling_group_by_payload( - ctx: Arc, - operator: Operator, - location_prefix: &str, - partitioned_payload: PartitionedPayload, -) -> Result>> { - let unique_name = GlobalUniqName::unique(); - let location = format!("{}/{}", location_prefix, unique_name); - - let mut write_size = 0; - let partition_count = partitioned_payload.partition_count(); - let mut write_data = Vec::with_capacity(partition_count); - let mut spilled_buckets_payloads = Vec::with_capacity(partition_count); - let mut rows = 0; - - for (bucket, payload) in partitioned_payload.payloads.into_iter().enumerate() { - if payload.len() == 0 { - continue; - } - - let data_block = payload.group_by_flush_all()?; - rows += data_block.num_rows(); - - let begin = write_size; - let columns = data_block.columns().to_vec(); - let mut columns_data = Vec::with_capacity(columns.len()); - let mut columns_layout = Vec::with_capacity(columns.len()); - for column in columns.into_iter() { - let column = column - .value - .convert_to_full_column(&column.data_type, data_block.num_rows()); - let column_data = serialize_column(&column); - write_size += column_data.len() as u64; - columns_layout.push(column_data.len() as u64); - columns_data.push(column_data); - } - - write_data.push(columns_data); - spilled_buckets_payloads.push(BucketSpilledPayload { - bucket: bucket as isize, - location: location.clone(), - data_range: begin..write_size, - columns_layout, - max_partition_count: partition_count, - }); - } - - Ok(Box::pin(async move { - let instant = Instant::now(); - - let mut write_bytes = 0; - if !write_data.is_empty() { - let mut writer = operator - .writer_with(&location) - .chunk(8 * 1024 * 1024) - .await?; - for write_bucket_data in write_data.into_iter() { - for data in write_bucket_data.into_iter() { - write_bytes += data.len(); - writer.write(data).await?; - } - } - - writer.close().await?; - } - - // perf - { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - write_bytes, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - instant.elapsed().as_millis() as usize, - ); - } - - { - let progress_val = ProgressValues { - rows, - bytes: write_bytes, - }; - ctx.get_group_by_spill_progress().incr(&progress_val); - } - - info!( - "Write aggregate spill {} successfully, elapsed: {:?}", - location, - instant.elapsed() - ); - - Ok(DataBlock::empty_with_meta( - AggregateMeta::::create_spilled(spilled_buckets_payloads), - )) - })) -} - -pub fn spilling_group_by_payload( - ctx: Arc, - operator: Operator, - method: &Method, - location_prefix: &str, - mut payload: HashTablePayload, ()>, -) -> Result>> { - let unique_name = GlobalUniqName::unique(); - let location = format!("{}/{}", location_prefix, unique_name); - - let mut write_size = 0; - let mut write_data = Vec::with_capacity(256); - let mut spilled_buckets_payloads = Vec::with_capacity(256); - let mut rows = 0; - for (bucket, inner_table) in payload.cell.hashtable.iter_tables_mut().enumerate() { - if inner_table.len() == 0 { - continue; - } - - let data_block = serialize_group_by(method, inner_table)?; - rows += data_block.num_rows(); - - let begin = write_size; - let columns = data_block.columns().to_vec(); - let mut columns_data = Vec::with_capacity(columns.len()); - let mut columns_layout = Vec::with_capacity(columns.len()); - for column in columns.into_iter() { - let column = column.to_column(data_block.num_rows()); - let column_data = serialize_column(&column); - write_size += column_data.len() as u64; - columns_layout.push(column_data.len() as u64); - columns_data.push(column_data); - } - - write_data.push(columns_data); - spilled_buckets_payloads.push(BucketSpilledPayload { - bucket: bucket as isize, - location: location.clone(), - data_range: begin..write_size, - columns_layout, - max_partition_count: 0, - }); - } - - Ok(Box::pin(async move { - let instant = Instant::now(); - - let mut write_bytes = 0; - if !write_data.is_empty() { - let mut writer = operator - .writer_with(&location) - .chunk(8 * 1024 * 1024) - .await?; - for write_bucket_data in write_data.into_iter() { - for data in write_bucket_data.into_iter() { - write_bytes += data.len(); - writer.write(data).await?; - } - } - - writer.close().await?; - } - - // perf - { - Profile::record_usize_profile(ProfileStatisticsName::RemoteSpillWriteCount, 1); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteBytes, - write_bytes, - ); - Profile::record_usize_profile( - ProfileStatisticsName::RemoteSpillWriteTime, - instant.elapsed().as_millis() as usize, - ); - } - - { - let progress_val = ProgressValues { - rows, - bytes: write_bytes, - }; - ctx.get_group_by_spill_progress().incr(&progress_val); - } - - info!( - "Write aggregate spill {} successfully, elapsed: {:?}", - location, - instant.elapsed() - ); - - Ok(DataBlock::empty_with_meta( - AggregateMeta::::create_spilled(spilled_buckets_payloads), - )) - })) -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs index 2d1ffd3a0a01..fd03b09e2f3f 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/serde/transform_spill_reader.rs @@ -39,25 +39,22 @@ use tokio::sync::Semaphore; use crate::pipelines::processors::transforms::aggregator::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::BucketSpilledPayload; use crate::pipelines::processors::transforms::aggregator::SerializedPayload; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -type DeserializingMeta = (AggregateMeta, VecDeque>); +type DeserializingMeta = (AggregateMeta, VecDeque>); -pub struct TransformSpillReader { +pub struct TransformSpillReader { input: Arc, output: Arc, operator: Operator, semaphore: Arc, deserialized_meta: Option, - reading_meta: Option>, - deserializing_meta: Option>, + reading_meta: Option, + deserializing_meta: Option, } #[async_trait::async_trait] -impl Processor - for TransformSpillReader -{ +impl Processor for TransformSpillReader { fn name(&self) -> String { String::from("TransformSpillReader") } @@ -98,12 +95,12 @@ impl Processor if let Some(block_meta) = data_block .get_meta() - .and_then(AggregateMeta::::downcast_ref_from) + .and_then(AggregateMeta::downcast_ref_from) { if matches!(block_meta, AggregateMeta::BucketSpilled(_)) { self.input.set_not_need_data(); let block_meta = data_block.take_meta().unwrap(); - self.reading_meta = AggregateMeta::::downcast_from(block_meta); + self.reading_meta = AggregateMeta::downcast_from(block_meta); return Ok(Event::Async); } @@ -114,7 +111,7 @@ impl Processor { self.input.set_not_need_data(); let block_meta = data_block.take_meta().unwrap(); - self.reading_meta = AggregateMeta::::downcast_from(block_meta); + self.reading_meta = AggregateMeta::downcast_from(block_meta); return Ok(Event::Async); } } @@ -137,10 +134,8 @@ impl Processor if let Some((meta, mut read_data)) = self.deserializing_meta.take() { match meta { AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), AggregateMeta::AggregatePayload(_) => unreachable!(), AggregateMeta::AggregateSpilling(_) => unreachable!(), - AggregateMeta::HashTable(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), AggregateMeta::BucketSpilled(payload) => { debug_assert!(read_data.len() == 1); @@ -164,9 +159,8 @@ impl Processor new_data.push(meta); } - self.deserialized_meta = Some(AggregateMeta::::create_partitioned( - bucket, new_data, - )); + self.deserialized_meta = + Some(AggregateMeta::create_partitioned(bucket, new_data)); } } } @@ -179,8 +173,6 @@ impl Processor if let Some(block_meta) = self.reading_meta.take() { match &block_meta { AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), - AggregateMeta::HashTable(_) => unreachable!(), AggregateMeta::AggregatePayload(_) => unreachable!(), AggregateMeta::AggregateSpilling(_) => unreachable!(), AggregateMeta::Serialized(_) => unreachable!(), @@ -284,17 +276,14 @@ impl Processor } } -impl TransformSpillReader { +impl TransformSpillReader { pub fn create( input: Arc, output: Arc, operator: Operator, semaphore: Arc, ) -> Result { - Ok(ProcessorPtr::create(Box::new(TransformSpillReader::< - Method, - V, - > { + Ok(ProcessorPtr::create(Box::new(TransformSpillReader { input, output, operator, @@ -305,7 +294,7 @@ impl TransformSpillReader) -> AggregateMeta { + fn deserialize(payload: BucketSpilledPayload, data: Vec) -> AggregateMeta { let mut begin = 0; let mut columns = Vec::with_capacity(payload.columns_layout.len()); @@ -314,7 +303,7 @@ impl TransformSpillReader::Serialized(SerializedPayload { + AggregateMeta::Serialized(SerializedPayload { bucket: payload.bucket, data_block: DataBlock::new_from_columns(columns), max_partition_count: payload.max_partition_count, @@ -322,5 +311,4 @@ impl TransformSpillReader = TransformSpillReader; -pub type TransformAggregateSpillReader = TransformSpillReader; +pub type TransformAggregateSpillReader = TransformSpillReader; diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs index 23598d0515c5..2a719a23c23d 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_final.rs @@ -12,61 +12,46 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::borrow::BorrowMut; use std::sync::Arc; use bumpalo::Bump; -use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::AggregateHashTable; -use databend_common_expression::ColumnBuilder; use databend_common_expression::DataBlock; use databend_common_expression::HashTableConfig; use databend_common_expression::PayloadFlushState; -use databend_common_functions::aggregates::StateAddr; -use databend_common_hashtable::HashtableEntryMutRefLike; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::BlockMetaTransform; use databend_common_pipeline_transforms::processors::BlockMetaTransformer; -use crate::pipelines::processors::transforms::aggregator::aggregate_cell::AggregateHashTableDropper; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::estimated_key_size; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::HashTableCell; -use crate::pipelines::processors::transforms::group_by::GroupColumnsBuilder; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::KeysColumnIter; -pub struct TransformFinalAggregate { - method: Method, +pub struct TransformFinalAggregate { params: Arc, flush_state: PayloadFlushState, } -impl TransformFinalAggregate { +impl TransformFinalAggregate { pub fn try_create( input: Arc, output: Arc, - method: Method, + params: Arc, ) -> Result> { Ok(BlockMetaTransformer::create( input, output, - TransformFinalAggregate:: { - method, + TransformFinalAggregate { params, flush_state: PayloadFlushState::default(), }, )) } - fn transform_agg_hashtable(&mut self, meta: AggregateMeta) -> Result { + fn transform_agg_hashtable(&mut self, meta: AggregateMeta) -> Result { let mut agg_hashtable: Option = None; if let AggregateMeta::Partitioned { bucket, data } = meta { for bucket_data in data { @@ -142,167 +127,10 @@ impl TransformFinalAggregate { } } -impl BlockMetaTransform> for TransformFinalAggregate -where Method: HashMethodBounds -{ +impl BlockMetaTransform for TransformFinalAggregate { const NAME: &'static str = "TransformFinalAggregate"; - fn transform(&mut self, meta: AggregateMeta) -> Result> { - if self.params.enable_experimental_aggregate_hashtable { - return Ok(vec![self.transform_agg_hashtable(meta)?]); - } - - if let AggregateMeta::Partitioned { bucket, data } = meta { - let arena = Arc::new(Bump::new()); - let hashtable = self.method.create_hash_table::(arena)?; - let _dropper = AggregateHashTableDropper::create(self.params.clone()); - let mut hash_cell = HashTableCell::::create(hashtable, _dropper); - - for bucket_data in data { - match bucket_data { - AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::Serialized(payload) => { - debug_assert!(bucket == payload.bucket); - - let aggregate_function_len = self.params.aggregate_functions.len(); - - let column = payload.get_group_by_column(); - let keys_iter = self.method.keys_iter_from_column(column)?; - - // first state places of current block - let places = { - let keys_iter = keys_iter.iter(); - let (len, _) = keys_iter.size_hint(); - let mut places = Vec::with_capacity(len); - - unsafe { - for key in keys_iter { - match hash_cell.hashtable.insert_and_entry(key) { - Ok(mut entry) => { - let place = - self.params.alloc_layout(&mut hash_cell.arena); - places.push(place); - - *entry.get_mut() = place.addr(); - } - Err(entry) => { - let place = Into::::into(*entry.get()); - places.push(place); - } - } - } - } - - places - }; - - let states_columns = (0..aggregate_function_len) - .map(|i| payload.data_block.get_by_offset(i)) - .collect::>(); - let mut states_binary_columns = Vec::with_capacity(states_columns.len()); - - for agg in states_columns.iter().take(aggregate_function_len) { - let col = agg.value.as_column().unwrap(); - states_binary_columns.push(col.slice(0..places.len())); - } - - let aggregate_functions = &self.params.aggregate_functions; - let offsets_aggregate_states = &self.params.offsets_aggregate_states; - - for (idx, aggregate_function) in aggregate_functions.iter().enumerate() { - aggregate_function.batch_merge( - &places, - offsets_aggregate_states[idx], - &states_binary_columns[idx], - )?; - } - } - AggregateMeta::HashTable(payload) => unsafe { - debug_assert!(bucket == payload.bucket); - - let aggregate_functions = &self.params.aggregate_functions; - let offsets_aggregate_states = &self.params.offsets_aggregate_states; - - for entry in payload.cell.hashtable.iter() { - let place = match hash_cell.hashtable.insert(entry.key()) { - Err(place) => StateAddr::new(*place), - Ok(entry) => { - let place = self.params.alloc_layout(&mut hash_cell.arena); - entry.write(place.addr()); - place - } - }; - - let old_place = StateAddr::new(*entry.get()); - for (idx, aggregate_function) in aggregate_functions.iter().enumerate() - { - let final_place = place.next(offsets_aggregate_states[idx]); - let state_place = old_place.next(offsets_aggregate_states[idx]); - aggregate_function.merge_states(final_place, state_place)?; - } - } - }, - AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), - } - } - - let keys_len = hash_cell.hashtable.len(); - let value_size = estimated_key_size(&hash_cell.hashtable); - - let mut group_columns_builder = - self.method - .group_columns_builder(keys_len, value_size, &self.params); - - let aggregate_functions = &self.params.aggregate_functions; - let offsets_aggregate_states = &self.params.offsets_aggregate_states; - - let mut aggregates_column_builder = { - let mut values = vec![]; - for aggregate_function in aggregate_functions { - let data_type = aggregate_function.return_type()?; - let builder = ColumnBuilder::with_capacity(&data_type, keys_len); - values.push(builder) - } - values - }; - - let mut places = Vec::with_capacity(keys_len); - for group_entity in hash_cell.hashtable.iter() { - places.push(StateAddr::new(*group_entity.get())); - group_columns_builder.append_value(group_entity.key()); - } - - for (idx, aggregate_function) in aggregate_functions.iter().enumerate() { - let builder = aggregates_column_builder[idx].borrow_mut(); - - if idx > 0 { - for place in places.iter_mut() { - *place = place.next( - offsets_aggregate_states[idx] - offsets_aggregate_states[idx - 1], - ); - } - } - aggregate_function.batch_merge_result(&places, 0, builder)?; - } - - // Build final state block. - let mut columns = aggregates_column_builder - .into_iter() - .map(|builder| builder.build()) - .collect::>(); - - let group_columns = group_columns_builder.finish()?; - columns.extend_from_slice(&group_columns); - - return Ok(vec![DataBlock::new_from_columns(columns)]); - } - - Err(ErrorCode::Internal( - "TransformFinalAggregate only recv AggregateMeta::Partitioned", - )) + fn transform(&mut self, meta: AggregateMeta) -> Result> { + Ok(vec![self.transform_agg_hashtable(meta)?]) } } diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs index cbd229d9b7ff..481b7d092c2c 100644 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs +++ b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_aggregate_partial.rs @@ -31,42 +31,28 @@ use databend_common_expression::HashTableConfig; use databend_common_expression::InputColumns; use databend_common_expression::PayloadFlushState; use databend_common_expression::ProbeState; -use databend_common_functions::aggregates::StateAddr; -use databend_common_functions::aggregates::StateAddrs; -use databend_common_hashtable::HashtableEntryMutRefLike; -use databend_common_hashtable::HashtableLike; use databend_common_pipeline_core::processors::InputPort; use databend_common_pipeline_core::processors::OutputPort; use databend_common_pipeline_core::processors::Processor; use databend_common_pipeline_transforms::processors::AccumulatingTransform; use databend_common_pipeline_transforms::processors::AccumulatingTransformer; -use log::info; -use crate::pipelines::processors::transforms::aggregator::aggregate_cell::AggregateHashTableDropper; use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::HashTableCell; -use crate::pipelines::processors::transforms::aggregator::PartitionedHashTableDropper; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; -use crate::pipelines::processors::transforms::group_by::PolymorphicKeysHelper; use crate::sessions::QueryContext; #[allow(clippy::enum_variant_names)] -enum HashTable { +enum HashTable { MovedOut, - HashTable(HashTableCell), AggregateHashTable(AggregateHashTable), - PartitionedHashTable(HashTableCell, usize>), } -impl Default for HashTable { +impl Default for HashTable { fn default() -> Self { Self::MovedOut } } struct AggregateSettings { - convert_threshold: usize, max_memory_usage: usize, spilling_bytes_threshold_per_proc: usize, } @@ -77,7 +63,6 @@ impl TryFrom> for AggregateSettings { fn try_from(ctx: Arc) -> std::result::Result { let settings = ctx.get_settings(); let max_threads = settings.get_max_threads()? as usize; - let convert_threshold = settings.get_group_by_two_level_threshold()? as usize; let mut memory_ratio = settings.get_aggregate_spilling_memory_ratio()? as f64 / 100_f64; if memory_ratio > 1_f64 { @@ -93,7 +78,6 @@ impl TryFrom> for AggregateSettings { }; Ok(AggregateSettings { - convert_threshold, max_memory_usage, spilling_bytes_threshold_per_proc: match settings .get_aggregate_spilling_bytes_threshold_per_proc()? @@ -106,10 +90,9 @@ impl TryFrom> for AggregateSettings { } // SELECT column_name, agg(xxx) FROM table_name GROUP BY column_name -pub struct TransformPartialAggregate { - method: Method, +pub struct TransformPartialAggregate { settings: AggregateSettings, - hash_table: HashTable, + hash_table: HashTable, probe_state: ProbeState, params: Arc, start: Instant, @@ -118,28 +101,15 @@ pub struct TransformPartialAggregate { processed_rows: usize, } -impl TransformPartialAggregate { +impl TransformPartialAggregate { pub fn try_create( ctx: Arc, - method: Method, input: Arc, output: Arc, params: Arc, config: HashTableConfig, ) -> Result> { - let hash_table = if !params.enable_experimental_aggregate_hashtable { - let arena = Arc::new(Bump::new()); - let hashtable = method.create_hash_table(arena)?; - let _dropper = AggregateHashTableDropper::create(params.clone()); - let hashtable = HashTableCell::create(hashtable, _dropper); - - match !Method::SUPPORT_PARTITIONED || !params.has_distinct_combinator() { - true => HashTable::HashTable(hashtable), - false => HashTable::PartitionedHashTable(PartitionedHashMethod::convert_hashtable( - &method, hashtable, - )?), - } - } else { + let hash_table = { let arena = Arc::new(Bump::new()); match !params.has_distinct_combinator() { true => HashTable::AggregateHashTable(AggregateHashTable::new( @@ -163,8 +133,7 @@ impl TransformPartialAggregate { Ok(AccumulatingTransformer::create( input, output, - TransformPartialAggregate:: { - method, + TransformPartialAggregate { params, hash_table, probe_state: ProbeState::default(), @@ -190,57 +159,6 @@ impl TransformPartialAggregate { } #[inline(always)] - #[allow(clippy::ptr_arg)] // &[StateAddr] slower than &StateAddrs ~20% - fn execute( - params: &Arc, - block: &DataBlock, - places: &StateAddrs, - ) -> Result<()> { - let AggregatorParams { - aggregate_functions, - offsets_aggregate_states, - aggregate_functions_arguments, - .. - } = &**params; - - // This can beneficial for the case of dereferencing - // This will help improve the performance ~hundreds of megabits per second - let aggr_arg_columns = Self::aggregate_arguments(block, aggregate_functions_arguments); - let aggr_arg_columns = aggr_arg_columns.as_slice(); - let rows = block.num_rows(); - for index in 0..aggregate_functions.len() { - let function = &aggregate_functions[index]; - function.accumulate_keys( - places, - offsets_aggregate_states[index], - aggr_arg_columns[index], - rows, - )?; - } - - Ok(()) - } - - #[inline(always)] - #[allow(clippy::ptr_arg)] // &[StateAddr] slower than &StateAddrs ~20% - fn execute_agg_index_block(&self, block: &DataBlock, places: &StateAddrs) -> Result<()> { - let aggregate_functions = &self.params.aggregate_functions; - let offsets_aggregate_states = &self.params.offsets_aggregate_states; - - let num_rows = block.num_rows(); - for index in 0..aggregate_functions.len() { - // Aggregation states are in the back of the block. - let agg_index = block.num_columns() - aggregate_functions.len() + index; - let function = &aggregate_functions[index]; - let offset = offsets_aggregate_states[index]; - let agg_state = block.get_by_offset(agg_index).to_column(num_rows); - - function.batch_merge(places, offset, &agg_state)?; - } - - Ok(()) - } - fn execute_one_block(&mut self, block: DataBlock) -> Result<()> { let is_agg_index_block = block .get_meta() @@ -261,48 +179,6 @@ impl TransformPartialAggregate { { match &mut self.hash_table { HashTable::MovedOut => unreachable!(), - HashTable::HashTable(hashtable) => { - let state = self.method.build_keys_state(group_columns, rows_num)?; - let mut places = Vec::with_capacity(rows_num); - - for key in self.method.build_keys_iter(&state)? { - places.push(match unsafe { hashtable.hashtable.insert_and_entry(key) } { - Err(entry) => Into::::into(*entry.get()), - Ok(mut entry) => { - let place = self.params.alloc_layout(&mut hashtable.arena); - *entry.get_mut() = place.addr(); - place - } - }) - } - - if is_agg_index_block { - self.execute_agg_index_block(&block, &places) - } else { - Self::execute(&self.params, &block, &places) - } - } - HashTable::PartitionedHashTable(hashtable) => { - let state = self.method.build_keys_state(group_columns, rows_num)?; - let mut places = Vec::with_capacity(rows_num); - - for key in self.method.build_keys_iter(&state)? { - places.push(match unsafe { hashtable.hashtable.insert_and_entry(key) } { - Err(entry) => Into::::into(*entry.get()), - Ok(mut entry) => { - let place = self.params.alloc_layout(&mut hashtable.arena); - *entry.get_mut() = place.addr(); - place - } - }) - } - - if is_agg_index_block { - self.execute_agg_index_block(&block, &places) - } else { - Self::execute(&self.params, &block, &places) - } - } HashTable::AggregateHashTable(hashtable) => { let (params_columns, states_index) = if is_agg_index_block { let num_columns = block.num_columns(); @@ -341,56 +217,14 @@ impl TransformPartialAggregate { } } -impl AccumulatingTransform for TransformPartialAggregate { +impl AccumulatingTransform for TransformPartialAggregate { const NAME: &'static str = "TransformPartialAggregate"; fn transform(&mut self, block: DataBlock) -> Result> { self.execute_one_block(block)?; - let is_new_agg = self.params.enable_experimental_aggregate_hashtable; - #[allow(clippy::collapsible_if)] - if Method::SUPPORT_PARTITIONED { - if !is_new_agg - && (matches!(&self.hash_table, HashTable::HashTable(cell) - if cell.len() >= self.settings.convert_threshold || - cell.allocated_bytes() >= self.settings.spilling_bytes_threshold_per_proc || - GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) - { - if let HashTable::HashTable(cell) = std::mem::take(&mut self.hash_table) { - self.hash_table = HashTable::PartitionedHashTable( - PartitionedHashMethod::convert_hashtable(&self.method, cell)?, - ); - } - } - - if !is_new_agg - && (matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) - || GLOBAL_MEM_STAT.get_memory_usage() as usize - >= self.settings.max_memory_usage) - { - if let HashTable::PartitionedHashTable(v) = std::mem::take(&mut self.hash_table) { - let _dropper = v._dropper.clone(); - let blocks = vec![DataBlock::empty_with_meta( - AggregateMeta::::create_spilling(v), - )]; - - let arena = Arc::new(Bump::new()); - let method = PartitionedHashMethod::::create(self.method.clone()); - let new_hashtable = method.create_hash_table(arena)?; - self.hash_table = HashTable::PartitionedHashTable(HashTableCell::create( - new_hashtable, - _dropper.unwrap(), - )); - return Ok(blocks); - } - - unreachable!() - } - } - - if is_new_agg - && (matches!(&self.hash_table, HashTable::AggregateHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc - || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) + if matches!(&self.hash_table, HashTable::AggregateHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc + || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage) { if let HashTable::AggregateHashTable(v) = std::mem::take(&mut self.hash_table) { let group_types = v.payload.group_types.clone(); @@ -409,7 +243,7 @@ impl AccumulatingTransform for TransformPartialAggrega .repartition(1 << config.max_radix_bits, &mut state); let blocks = vec![DataBlock::empty_with_meta( - AggregateMeta::::create_agg_spilling(partitioned_payload), + AggregateMeta::create_agg_spilling(partitioned_payload), )]; let arena = Arc::new(Bump::new()); @@ -434,33 +268,6 @@ impl AccumulatingTransform for TransformPartialAggrega true => vec![], false => unreachable!(), }, - HashTable::HashTable(v) => match v.hashtable.len() == 0 { - true => vec![], - false => { - vec![DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(-1, v), - )] - } - }, - HashTable::PartitionedHashTable(v) => { - info!( - "Processed {} different keys, allocated {} memory while in group by.", - convert_number_size(v.len() as f64), - convert_byte_size(v.allocated_bytes() as f64) - ); - - let cells = PartitionedHashTableDropper::split_cell(v); - let mut blocks = Vec::with_capacity(cells.len()); - for (bucket, cell) in cells.into_iter().enumerate() { - if cell.hashtable.len() != 0 { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(bucket as isize, cell), - )); - } - } - - blocks - } HashTable::AggregateHashTable(hashtable) => { let partition_count = hashtable.payload.partition_count(); let mut blocks = Vec::with_capacity(partition_count); @@ -487,7 +294,7 @@ impl AccumulatingTransform for TransformPartialAggrega for (bucket, payload) in hashtable.payload.payloads.into_iter().enumerate() { if payload.len() != 0 { blocks.push(DataBlock::empty_with_meta( - AggregateMeta::::create_agg_payload( + AggregateMeta::create_agg_payload( bucket as isize, payload, partition_count, diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs deleted file mode 100644 index 2065d6e7c4bb..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_final.rs +++ /dev/null @@ -1,201 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::sync::Arc; - -use bumpalo::Bump; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::AggregateHashTable; -use databend_common_expression::DataBlock; -use databend_common_expression::HashTableConfig; -use databend_common_expression::PayloadFlushState; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::Processor; -use databend_common_pipeline_transforms::processors::BlockMetaTransform; -use databend_common_pipeline_transforms::processors::BlockMetaTransformer; - -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::estimated_key_size; -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::group_by::GroupColumnsBuilder; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::KeysColumnIter; - -pub struct TransformFinalGroupBy { - method: Method, - params: Arc, - flush_state: PayloadFlushState, -} - -impl TransformFinalGroupBy { - pub fn try_create( - input: Arc, - output: Arc, - method: Method, - params: Arc, - ) -> Result> { - Ok(BlockMetaTransformer::create( - input, - output, - TransformFinalGroupBy:: { - method, - params, - flush_state: PayloadFlushState::default(), - }, - )) - } - - fn transform_agg_hashtable(&mut self, meta: AggregateMeta) -> Result { - let mut agg_hashtable: Option = None; - if let AggregateMeta::Partitioned { bucket, data } = meta { - for bucket_data in data { - match bucket_data { - AggregateMeta::Serialized(payload) => match agg_hashtable.as_mut() { - Some(ht) => { - debug_assert!(bucket == payload.bucket); - let payload = payload.convert_to_partitioned_payload( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - 0, - Arc::new(Bump::new()), - )?; - ht.combine_payloads(&payload, &mut self.flush_state)?; - } - None => { - debug_assert!(bucket == payload.bucket); - agg_hashtable = Some(payload.convert_to_aggregate_table( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - 0, - Arc::new(Bump::new()), - true, - )?); - } - }, - AggregateMeta::AggregatePayload(payload) => match agg_hashtable.as_mut() { - Some(ht) => { - debug_assert!(bucket == payload.bucket); - ht.combine_payload(&payload.payload, &mut self.flush_state)?; - } - None => { - debug_assert!(bucket == payload.bucket); - let capacity = - AggregateHashTable::get_capacity_for_count(payload.payload.len()); - let mut hashtable = AggregateHashTable::new_with_capacity( - self.params.group_data_types.clone(), - self.params.aggregate_functions.clone(), - HashTableConfig::default().with_initial_radix_bits(0), - capacity, - Arc::new(Bump::new()), - ); - hashtable.combine_payload(&payload.payload, &mut self.flush_state)?; - agg_hashtable = Some(hashtable); - } - }, - _ => unreachable!(), - } - } - } - - if let Some(mut ht) = agg_hashtable { - let mut blocks = vec![]; - self.flush_state.clear(); - - loop { - if ht.merge_result(&mut self.flush_state)? { - let cols = self.flush_state.take_group_columns(); - blocks.push(DataBlock::new_from_columns(cols)); - } else { - break; - } - } - - if blocks.is_empty() { - return Ok(self.params.empty_result_block()); - } - - return DataBlock::concat(&blocks); - } - Ok(self.params.empty_result_block()) - } -} - -impl BlockMetaTransform> for TransformFinalGroupBy -where Method: HashMethodBounds -{ - const NAME: &'static str = "TransformFinalGroupBy"; - - fn transform(&mut self, meta: AggregateMeta) -> Result> { - if self.params.enable_experimental_aggregate_hashtable { - return Ok(vec![self.transform_agg_hashtable(meta)?]); - } - - if let AggregateMeta::Partitioned { bucket, data } = meta { - let arena = Arc::new(Bump::new()); - let mut hashtable = self.method.create_hash_table::<()>(arena)?; - - for bucket_data in data { - match bucket_data { - AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::Serialized(payload) => { - debug_assert!(bucket == payload.bucket); - let column = payload.get_group_by_column(); - let keys_iter = self.method.keys_iter_from_column(column)?; - - unsafe { - for key in keys_iter.iter() { - let _ = hashtable.insert_and_entry(key); - } - } - } - AggregateMeta::HashTable(payload) => unsafe { - debug_assert!(bucket == payload.bucket); - - for key in payload.cell.hashtable.iter() { - let _ = hashtable.insert_and_entry(key.key()); - } - }, - AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), - } - } - - let value_size = estimated_key_size(&hashtable); - let keys_len = hashtable.len(); - - let mut group_columns_builder = - self.method - .group_columns_builder(keys_len, value_size, &self.params); - - for group_entity in hashtable.iter() { - group_columns_builder.append_value(group_entity.key()); - } - - return Ok(vec![DataBlock::new_from_columns( - group_columns_builder.finish()?, - )]); - } - - Err(ErrorCode::Internal( - "TransformFinalGroupBy only recv AggregateMeta::Partitioned", - )) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs deleted file mode 100644 index 7878df9e5ef8..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_group_by_partial.rs +++ /dev/null @@ -1,360 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::sync::Arc; -use std::time::Instant; -use std::vec; - -use bumpalo::Bump; -use databend_common_base::base::convert_byte_size; -use databend_common_base::base::convert_number_size; -use databend_common_base::runtime::GLOBAL_MEM_STAT; -use databend_common_catalog::table_context::TableContext; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::AggregateHashTable; -use databend_common_expression::DataBlock; -use databend_common_expression::HashTableConfig; -use databend_common_expression::InputColumns; -use databend_common_expression::PayloadFlushState; -use databend_common_expression::ProbeState; -use databend_common_hashtable::HashtableLike; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::Processor; -use databend_common_pipeline_transforms::processors::AccumulatingTransform; -use databend_common_pipeline_transforms::processors::AccumulatingTransformer; -use log::info; - -use crate::pipelines::processors::transforms::aggregator::aggregate_cell::GroupByHashTableDropper; -use crate::pipelines::processors::transforms::aggregator::aggregate_cell::HashTableCell; -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::PartitionedHashTableDropper; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; -use crate::pipelines::processors::transforms::group_by::PolymorphicKeysHelper; -use crate::sessions::QueryContext; - -#[allow(clippy::enum_variant_names)] -enum HashTable { - MovedOut, - HashTable(HashTableCell), - AggregateHashTable(AggregateHashTable), - PartitionedHashTable(HashTableCell, ()>), -} - -impl Default for HashTable { - fn default() -> Self { - Self::MovedOut - } -} - -struct GroupBySettings { - convert_threshold: usize, - max_memory_usage: usize, - spilling_bytes_threshold_per_proc: usize, -} - -impl TryFrom> for GroupBySettings { - type Error = ErrorCode; - - fn try_from(ctx: Arc) -> std::result::Result { - let settings = ctx.get_settings(); - let max_threads = settings.get_max_threads()? as usize; - let convert_threshold = settings.get_group_by_two_level_threshold()? as usize; - let mut memory_ratio = settings.get_aggregate_spilling_memory_ratio()? as f64 / 100_f64; - - if memory_ratio > 1_f64 { - memory_ratio = 1_f64; - } - - let max_memory_usage = match settings.get_max_memory_usage()? { - 0 => usize::MAX, - max_memory_usage => match memory_ratio { - x if x == 0_f64 => usize::MAX, - memory_ratio => (max_memory_usage as f64 * memory_ratio) as usize, - }, - }; - - Ok(GroupBySettings { - max_memory_usage, - convert_threshold, - spilling_bytes_threshold_per_proc: match settings - .get_aggregate_spilling_bytes_threshold_per_proc()? - { - 0 => max_memory_usage / max_threads, - spilling_bytes_threshold_per_proc => spilling_bytes_threshold_per_proc, - }, - }) - } -} - -// SELECT column_name FROM table_name GROUP BY column_name -pub struct TransformPartialGroupBy { - method: Method, - hash_table: HashTable, - probe_state: ProbeState, - settings: GroupBySettings, - params: Arc, - - start: Instant, - first_block_start: Option, - processed_rows: usize, - processed_bytes: usize, -} - -impl TransformPartialGroupBy { - pub fn try_create( - ctx: Arc, - method: Method, - input: Arc, - output: Arc, - params: Arc, - config: HashTableConfig, - ) -> Result> { - let hash_table = if !params.enable_experimental_aggregate_hashtable { - let arena = Arc::new(Bump::new()); - let hashtable = method.create_hash_table(arena)?; - let _dropper = GroupByHashTableDropper::::create(); - HashTable::HashTable(HashTableCell::create(hashtable, _dropper)) - } else { - let arena = Arc::new(Bump::new()); - HashTable::AggregateHashTable(AggregateHashTable::new( - params.group_data_types.clone(), - params.aggregate_functions.clone(), - config, - arena, - )) - }; - - Ok(AccumulatingTransformer::create( - input, - output, - TransformPartialGroupBy:: { - method, - hash_table, - probe_state: ProbeState::default(), - params, - settings: GroupBySettings::try_from(ctx)?, - start: Instant::now(), - first_block_start: None, - processed_bytes: 0, - processed_rows: 0, - }, - )) - } -} - -impl AccumulatingTransform for TransformPartialGroupBy { - const NAME: &'static str = "TransformPartialGroupBy"; - - fn transform(&mut self, block: DataBlock) -> Result> { - let block = block.consume_convert_to_full(); - - let rows_num = block.num_rows(); - - self.processed_bytes += block.memory_size(); - self.processed_rows += rows_num; - if self.first_block_start.is_none() { - self.first_block_start = Some(Instant::now()); - } - - let group_columns = InputColumns::new_block_proxy(&self.params.group_columns, &block); - - { - match &mut self.hash_table { - HashTable::MovedOut => unreachable!(), - HashTable::HashTable(cell) => { - let state = self.method.build_keys_state(group_columns, rows_num)?; - for key in self.method.build_keys_iter(&state)? { - unsafe { - let _ = cell.hashtable.insert_and_entry(key); - } - } - } - HashTable::PartitionedHashTable(cell) => { - let state = self.method.build_keys_state(group_columns, rows_num)?; - for key in self.method.build_keys_iter(&state)? { - unsafe { - let _ = cell.hashtable.insert_and_entry(key); - } - } - } - HashTable::AggregateHashTable(hashtable) => { - let _ = hashtable.add_groups( - &mut self.probe_state, - group_columns, - &[(&[]).into()], - (&[]).into(), - rows_num, - )?; - } - }; - - let is_new_agg = self.params.enable_experimental_aggregate_hashtable; - - #[allow(clippy::collapsible_if)] - if Method::SUPPORT_PARTITIONED { - if !is_new_agg - && (matches!(&self.hash_table, HashTable::HashTable(cell) - if cell.len() >= self.settings.convert_threshold || - cell.allocated_bytes() >= self.settings.spilling_bytes_threshold_per_proc || - GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) - { - if let HashTable::HashTable(cell) = std::mem::take(&mut self.hash_table) { - self.hash_table = HashTable::PartitionedHashTable( - PartitionedHashMethod::convert_hashtable(&self.method, cell)?, - ); - } - } - - if !is_new_agg - && (matches!(&self.hash_table, HashTable::PartitionedHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc) - || GLOBAL_MEM_STAT.get_memory_usage() as usize - >= self.settings.max_memory_usage) - { - if let HashTable::PartitionedHashTable(v) = std::mem::take(&mut self.hash_table) - { - let _dropper = v._dropper.clone(); - let blocks = vec![DataBlock::empty_with_meta( - AggregateMeta::::create_spilling(v), - )]; - - let arena = Arc::new(Bump::new()); - let method = PartitionedHashMethod::::create(self.method.clone()); - let new_hashtable = method.create_hash_table(arena)?; - self.hash_table = HashTable::PartitionedHashTable(HashTableCell::create( - new_hashtable, - _dropper.unwrap(), - )); - return Ok(blocks); - } - - unreachable!() - } - } - - if is_new_agg - && (matches!(&self.hash_table, HashTable::AggregateHashTable(cell) if cell.allocated_bytes() > self.settings.spilling_bytes_threshold_per_proc - || GLOBAL_MEM_STAT.get_memory_usage() as usize >= self.settings.max_memory_usage)) - { - if let HashTable::AggregateHashTable(v) = std::mem::take(&mut self.hash_table) { - let group_types = v.payload.group_types.clone(); - let aggrs = v.payload.aggrs.clone(); - v.config.update_current_max_radix_bits(); - let config = v - .config - .clone() - .with_initial_radix_bits(v.config.max_radix_bits); - let mut state = PayloadFlushState::default(); - - // repartition to max for normalization - let partitioned_payload = v - .payload - .repartition(1 << config.max_radix_bits, &mut state); - let blocks = vec![DataBlock::empty_with_meta( - AggregateMeta::::create_agg_spilling(partitioned_payload), - )]; - - let arena = Arc::new(Bump::new()); - self.hash_table = HashTable::AggregateHashTable(AggregateHashTable::new( - group_types, - aggrs, - config, - arena, - )); - return Ok(blocks); - } - - unreachable!() - } - } - - Ok(vec![]) - } - - fn on_finish(&mut self, output: bool) -> Result> { - Ok(match std::mem::take(&mut self.hash_table) { - HashTable::MovedOut => match !output && std::thread::panicking() { - true => vec![], - false => unreachable!(), - }, - HashTable::HashTable(cell) => match cell.hashtable.len() == 0 { - true => vec![], - false => { - vec![DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(-1, cell), - )] - } - }, - HashTable::PartitionedHashTable(v) => { - info!( - "Processed {} different keys, allocated {} memory while in group by.", - convert_number_size(v.len() as f64), - convert_byte_size(v.allocated_bytes() as f64) - ); - - let _ = v.hashtable.unsize_key_size(); - let cells = PartitionedHashTableDropper::split_cell(v); - let mut blocks = Vec::with_capacity(cells.len()); - for (bucket, cell) in cells.into_iter().enumerate() { - if cell.hashtable.len() != 0 { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(bucket as isize, cell), - )); - } - } - - blocks - } - HashTable::AggregateHashTable(hashtable) => { - let partition_count = hashtable.payload.partition_count(); - let mut blocks = Vec::with_capacity(partition_count); - - log::info!( - "Aggregated {} to {} rows in {} sec(real: {}). ({} rows/sec, {}/sec, {})", - self.processed_rows, - hashtable.payload.len(), - self.start.elapsed().as_secs_f64(), - if let Some(t) = &self.first_block_start { - t.elapsed().as_secs_f64() - } else { - self.start.elapsed().as_secs_f64() - }, - convert_number_size( - self.processed_rows as f64 / self.start.elapsed().as_secs_f64() - ), - convert_byte_size( - self.processed_bytes as f64 / self.start.elapsed().as_secs_f64() - ), - convert_byte_size(self.processed_bytes as f64), - ); - - for (bucket, payload) in hashtable.payload.payloads.into_iter().enumerate() { - if payload.len() != 0 { - blocks.push(DataBlock::empty_with_meta( - AggregateMeta::::create_agg_payload( - bucket as isize, - payload, - partition_count, - ), - )); - } - } - blocks - } - }) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs b/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs deleted file mode 100644 index a33be6038c1d..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/aggregator/transform_partition_bucket.rs +++ /dev/null @@ -1,557 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::any::Any; -use std::collections::btree_map::Entry; -use std::collections::BTreeMap; -use std::marker::PhantomData; -use std::mem::take; -use std::sync::Arc; - -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::BlockMetaInfoDowncast; -use databend_common_expression::DataBlock; -use databend_common_hashtable::hash2bucket; -use databend_common_hashtable::HashtableLike; -use databend_common_pipeline_core::processors::Event; -use databend_common_pipeline_core::processors::InputPort; -use databend_common_pipeline_core::processors::OutputPort; -use databend_common_pipeline_core::processors::Processor; -use databend_common_pipeline_core::processors::ProcessorPtr; -use databend_common_pipeline_core::Pipe; -use databend_common_pipeline_core::PipeItem; -use databend_common_pipeline_core::Pipeline; -use databend_common_storage::DataOperator; -use tokio::sync::Semaphore; - -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::AggregateMeta; -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::HashTablePayload; -use crate::pipelines::processors::transforms::aggregator::aggregate_meta::SerializedPayload; -use crate::pipelines::processors::transforms::aggregator::new_transform_partition_bucket::NewTransformPartitionBucket; -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::PartitionedHashTableDropper; -use crate::pipelines::processors::transforms::aggregator::TransformAggregateSpillReader; -use crate::pipelines::processors::transforms::aggregator::TransformFinalAggregate; -use crate::pipelines::processors::transforms::aggregator::TransformFinalGroupBy; -use crate::pipelines::processors::transforms::aggregator::TransformGroupBySpillReader; -use crate::pipelines::processors::transforms::group_by::HashMethodBounds; -use crate::pipelines::processors::transforms::group_by::KeysColumnIter; -use crate::pipelines::processors::transforms::group_by::PartitionedHashMethod; - -static SINGLE_LEVEL_BUCKET_NUM: isize = -1; - -struct InputPortState { - port: Arc, - bucket: isize, -} - -pub struct TransformPartitionBucket { - output: Arc, - inputs: Vec, - method: Method, - working_bucket: isize, - pushing_bucket: isize, - initialized_all_inputs: bool, - buckets_blocks: BTreeMap>, - unsplitted_blocks: Vec, - _phantom: PhantomData, -} - -impl - TransformPartitionBucket -{ - pub fn create(method: Method, input_nums: usize) -> Result { - let mut inputs = Vec::with_capacity(input_nums); - - for _index in 0..input_nums { - inputs.push(InputPortState { - bucket: -1, - port: InputPort::create(), - }); - } - - Ok(TransformPartitionBucket { - method, - inputs, - working_bucket: 0, - pushing_bucket: 0, - output: OutputPort::create(), - buckets_blocks: BTreeMap::new(), - unsplitted_blocks: vec![], - initialized_all_inputs: false, - _phantom: Default::default(), - }) - } - - pub fn get_inputs(&self) -> Vec> { - let mut inputs = Vec::with_capacity(self.inputs.len()); - - for input_state in &self.inputs { - inputs.push(input_state.port.clone()); - } - - inputs - } - - pub fn get_output(&self) -> Arc { - self.output.clone() - } - - fn initialize_all_inputs(&mut self) -> Result { - self.initialized_all_inputs = true; - - for index in 0..self.inputs.len() { - if self.inputs[index].port.is_finished() { - continue; - } - - // We pull the first unsplitted data block - if self.inputs[index].bucket > SINGLE_LEVEL_BUCKET_NUM { - continue; - } - - if !self.inputs[index].port.has_data() { - self.inputs[index].port.set_need_data(); - self.initialized_all_inputs = false; - continue; - } - - let data_block = self.inputs[index].port.pull_data().unwrap()?; - self.inputs[index].bucket = self.add_bucket(data_block)?; - - if self.inputs[index].bucket <= SINGLE_LEVEL_BUCKET_NUM { - self.inputs[index].port.set_need_data(); - self.initialized_all_inputs = false; - } - } - - Ok(self.initialized_all_inputs) - } - - fn add_bucket(&mut self, mut data_block: DataBlock) -> Result { - if let Some(block_meta) = data_block.get_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_ref_from(block_meta) { - let (bucket, res) = match block_meta { - AggregateMeta::Spilling(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), - AggregateMeta::BucketSpilled(payload) => { - (payload.bucket, SINGLE_LEVEL_BUCKET_NUM) - } - AggregateMeta::Serialized(payload) => (payload.bucket, payload.bucket), - AggregateMeta::HashTable(payload) => (payload.bucket, payload.bucket), - AggregateMeta::Spilled(_) => { - let meta = data_block.take_meta().unwrap(); - - if let Some(AggregateMeta::Spilled(buckets_payload)) = - AggregateMeta::::downcast_from(meta) - { - for bucket_payload in buckets_payload { - match self.buckets_blocks.entry(bucket_payload.bucket) { - Entry::Vacant(v) => { - v.insert(vec![DataBlock::empty_with_meta( - AggregateMeta::::create_bucket_spilled( - bucket_payload, - ), - )]); - } - Entry::Occupied(mut v) => { - v.get_mut().push(DataBlock::empty_with_meta( - AggregateMeta::::create_bucket_spilled( - bucket_payload, - ), - )); - } - }; - } - - return Ok(SINGLE_LEVEL_BUCKET_NUM); - } - - unreachable!() - } - }; - - if bucket > SINGLE_LEVEL_BUCKET_NUM { - match self.buckets_blocks.entry(bucket) { - Entry::Vacant(v) => { - v.insert(vec![data_block]); - } - Entry::Occupied(mut v) => { - v.get_mut().push(data_block); - } - }; - - return Ok(res); - } - } - } - - self.unsplitted_blocks.push(data_block); - Ok(SINGLE_LEVEL_BUCKET_NUM) - } - - fn try_push_data_block(&mut self) -> bool { - match self.buckets_blocks.is_empty() { - true => self.try_push_single_level(), - false => self.try_push_two_level(), - } - } - - fn try_push_two_level(&mut self) -> bool { - while self.pushing_bucket < self.working_bucket { - if let Some(bucket_blocks) = self.buckets_blocks.remove(&self.pushing_bucket) { - let data_block = Self::convert_blocks(self.pushing_bucket, bucket_blocks); - self.output.push_data(Ok(data_block)); - self.pushing_bucket += 1; - return true; - } - - self.pushing_bucket += 1; - } - - false - } - - fn try_push_single_level(&mut self) -> bool { - if !self.unsplitted_blocks.is_empty() { - let data_blocks = take(&mut self.unsplitted_blocks); - self.output.push_data(Ok(Self::convert_blocks( - SINGLE_LEVEL_BUCKET_NUM, - data_blocks, - ))); - return true; - } - - false - } - - fn convert_blocks(bucket: isize, data_blocks: Vec) -> DataBlock { - let mut data = Vec::with_capacity(data_blocks.len()); - for mut data_block in data_blocks.into_iter() { - if let Some(block_meta) = data_block.take_meta() { - if let Some(block_meta) = AggregateMeta::::downcast_from(block_meta) { - data.push(block_meta); - } - } - } - - DataBlock::empty_with_meta(AggregateMeta::::create_partitioned(bucket, data)) - } - - fn partition_block(&self, payload: SerializedPayload) -> Result>> { - let column = payload.get_group_by_column(); - let keys_iter = self.method.keys_iter_from_column(column)?; - - let mut indices = Vec::with_capacity(payload.data_block.num_rows()); - - for key_item in keys_iter.iter() { - let hash = self.method.get_hash(key_item); - indices.push(hash2bucket::<8, true>(hash as usize) as u16); - } - - let scatter_blocks = DataBlock::scatter(&payload.data_block, &indices, 1 << 8)?; - - let mut blocks = Vec::with_capacity(scatter_blocks.len()); - for (bucket, data_block) in scatter_blocks.into_iter().enumerate() { - blocks.push(match data_block.is_empty() { - true => None, - false => Some(DataBlock::empty_with_meta( - AggregateMeta::::create_serialized(bucket as isize, data_block, 0), - )), - }); - } - - Ok(blocks) - } - - fn partition_hashtable( - &self, - payload: HashTablePayload, - ) -> Result>> { - let temp = PartitionedHashMethod::convert_hashtable(&self.method, payload.cell)?; - let cells = PartitionedHashTableDropper::split_cell(temp); - - let mut data_blocks = Vec::with_capacity(cells.len()); - for (bucket, cell) in cells.into_iter().enumerate() { - data_blocks.push(match cell.hashtable.len() == 0 { - true => None, - false => Some(DataBlock::empty_with_meta( - AggregateMeta::::create_hashtable(bucket as isize, cell), - )), - }) - } - - Ok(data_blocks) - } -} - -#[async_trait::async_trait] -impl Processor - for TransformPartitionBucket -{ - fn name(&self) -> String { - String::from("TransformPartitionBucket") - } - - fn as_any(&mut self) -> &mut dyn Any { - self - } - - fn event(&mut self) -> Result { - if self.output.is_finished() { - for input_state in &self.inputs { - input_state.port.finish(); - } - - self.buckets_blocks.clear(); - return Ok(Event::Finished); - } - - // We pull the first unsplitted data block - if !self.initialized_all_inputs && !self.initialize_all_inputs()? { - return Ok(Event::NeedData); - } - - if !self.buckets_blocks.is_empty() && !self.unsplitted_blocks.is_empty() { - // Split data blocks if it's unsplitted. - return Ok(Event::Sync); - } - - if !self.output.can_push() { - for input_state in &self.inputs { - input_state.port.set_not_need_data(); - } - - return Ok(Event::NeedConsume); - } - - let pushed_data_block = self.try_push_data_block(); - - loop { - // Try to pull the next data or until the port is closed - let mut all_inputs_is_finished = true; - let mut all_port_prepared_data = true; - - for index in 0..self.inputs.len() { - if self.inputs[index].port.is_finished() { - continue; - } - - all_inputs_is_finished = false; - if self.inputs[index].bucket > self.working_bucket { - continue; - } - - if !self.inputs[index].port.has_data() { - all_port_prepared_data = false; - self.inputs[index].port.set_need_data(); - continue; - } - - let data_block = self.inputs[index].port.pull_data().unwrap()?; - self.inputs[index].bucket = self.add_bucket(data_block)?; - debug_assert!(self.unsplitted_blocks.is_empty()); - - if self.inputs[index].bucket <= self.working_bucket { - all_port_prepared_data = false; - self.inputs[index].port.set_need_data(); - } - } - - if all_inputs_is_finished { - break; - } - - if !all_port_prepared_data { - return Ok(Event::NeedData); - } - - self.working_bucket += 1; - } - - if pushed_data_block || self.try_push_data_block() { - return Ok(Event::NeedConsume); - } - - if let Some((bucket, bucket_blocks)) = self.buckets_blocks.pop_first() { - let data_block = Self::convert_blocks(bucket, bucket_blocks); - self.output.push_data(Ok(data_block)); - return Ok(Event::NeedConsume); - } - - self.output.finish(); - Ok(Event::Finished) - } - - fn process(&mut self) -> Result<()> { - let block_meta = self - .unsplitted_blocks - .pop() - .and_then(|mut block| block.take_meta()) - .and_then(AggregateMeta::::downcast_from); - - match block_meta { - None => Err(ErrorCode::Internal( - "Internal error, TransformPartitionBucket only recv AggregateMeta.", - )), - Some(agg_block_meta) => { - let data_blocks = match agg_block_meta { - AggregateMeta::Spilled(_) => unreachable!(), - AggregateMeta::BucketSpilled(_) => unreachable!(), - AggregateMeta::Spilling(_) => unreachable!(), - AggregateMeta::Partitioned { .. } => unreachable!(), - AggregateMeta::Serialized(payload) => self.partition_block(payload)?, - AggregateMeta::HashTable(payload) => self.partition_hashtable(payload)?, - AggregateMeta::AggregatePayload(_) => unreachable!(), - AggregateMeta::AggregateSpilling(_) => unreachable!(), - }; - - for (bucket, block) in data_blocks.into_iter().enumerate() { - if let Some(data_block) = block { - match self.buckets_blocks.entry(bucket as isize) { - Entry::Vacant(v) => { - v.insert(vec![data_block]); - } - Entry::Occupied(mut v) => { - v.get_mut().push(data_block); - } - }; - } - } - - Ok(()) - } - } - } -} - -pub fn build_partition_bucket( - method: Method, - pipeline: &mut Pipeline, - params: Arc, -) -> Result<()> { - if params.enable_experimental_aggregate_hashtable { - let input_nums = pipeline.output_len(); - let transform = - NewTransformPartitionBucket::::create(input_nums, params.clone())?; - - let output = transform.get_output(); - let inputs_port = transform.get_inputs(); - - pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - inputs_port, - vec![output], - )])); - - pipeline.try_resize(input_nums)?; - - let semaphore = Arc::new(Semaphore::new(params.max_spill_io_requests)); - let operator = DataOperator::instance().operator(); - pipeline.add_transform(|input, output| { - let operator = operator.clone(); - match params.aggregate_functions.is_empty() { - true => TransformGroupBySpillReader::::create( - input, - output, - operator, - semaphore.clone(), - ), - false => TransformAggregateSpillReader::::create( - input, - output, - operator, - semaphore.clone(), - ), - } - })?; - - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - match params.aggregate_functions.is_empty() { - true => TransformFinalGroupBy::try_create( - input, - output, - method.clone(), - params.clone(), - )?, - false => TransformFinalAggregate::try_create( - input, - output, - method.clone(), - params.clone(), - )?, - }, - )) - })?; - } else { - let input_nums = pipeline.output_len(); - let transform = TransformPartitionBucket::::create(method.clone(), input_nums)?; - - let output = transform.get_output(); - let inputs_port = transform.get_inputs(); - - pipeline.add_pipe(Pipe::create(inputs_port.len(), 1, vec![PipeItem::create( - ProcessorPtr::create(Box::new(transform)), - inputs_port, - vec![output], - )])); - - pipeline.try_resize(input_nums)?; - - let semaphore = Arc::new(Semaphore::new(128)); - let operator = DataOperator::instance().operator(); - pipeline.add_transform(|input, output| { - let operator = operator.clone(); - match params.aggregate_functions.is_empty() { - true => TransformGroupBySpillReader::::create( - input, - output, - operator, - semaphore.clone(), - ), - false => TransformAggregateSpillReader::::create( - input, - output, - operator, - semaphore.clone(), - ), - } - })?; - - pipeline.add_transform(|input, output| { - Ok(ProcessorPtr::create( - match params.aggregate_functions.is_empty() { - true => TransformFinalGroupBy::try_create( - input, - output, - method.clone(), - params.clone(), - )?, - false => TransformFinalAggregate::try_create( - input, - output, - method.clone(), - params.clone(), - )?, - }, - )) - })?; - } - - Ok(()) -} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_groups_builder.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_groups_builder.rs deleted file mode 100644 index 7e5dfaca922d..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_groups_builder.rs +++ /dev/null @@ -1,228 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::marker::PhantomData; - -use databend_common_exception::Result; -use databend_common_expression::types::binary::BinaryColumnBuilder; -use databend_common_expression::types::string::StringColumnBuilder; -use databend_common_expression::types::DataType; -use databend_common_expression::Column; -use databend_common_expression::ColumnBuilder; -use databend_common_expression::HashMethodFixedKeys; -use databend_common_hashtable::DictionaryKeys; - -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; - -pub trait GroupColumnsBuilder { - type T; - fn append_value(&mut self, v: Self::T); - fn finish(self) -> Result>; -} - -pub struct FixedKeysGroupColumnsBuilder<'a, T> { - _t: PhantomData<&'a ()>, - data: Vec, - group_column_indices: Vec, - group_data_types: Vec, -} - -impl FixedKeysGroupColumnsBuilder<'_, T> { - pub fn create(capacity: usize, params: &AggregatorParams) -> Self { - Self { - _t: Default::default(), - data: Vec::with_capacity(capacity), - group_column_indices: params.group_columns.clone(), - group_data_types: params.group_data_types.clone(), - } - } -} - -impl<'a, T: Copy + Send + Sync + 'static> GroupColumnsBuilder - for FixedKeysGroupColumnsBuilder<'a, T> -{ - type T = &'a T; - - #[inline] - fn append_value(&mut self, v: Self::T) { - self.data.push(*v); - } - - #[inline] - fn finish(self) -> Result> { - let method = HashMethodFixedKeys::::default(); - method.deserialize_group_columns( - self.data, - &self - .group_column_indices - .iter() - .cloned() - .zip(self.group_data_types.iter().cloned()) - .collect::>(), - ) - } -} - -pub struct SerializedKeysGroupColumnsBuilder<'a> { - data: Vec<&'a [u8]>, - group_data_types: Vec, - - single_binary_builder: Option, - single_string_builder: Option, -} - -impl SerializedKeysGroupColumnsBuilder<'_> { - pub fn create(capacity: usize, data_capacity: usize, params: &AggregatorParams) -> Self { - let (single_binary_builder, single_string_builder, data) = - if params.group_data_types.len() == 1 { - if params.group_data_types[0].is_string() { - ( - None, - Some(StringColumnBuilder::with_capacity(capacity)), - vec![], - ) - } else if params.group_data_types[0].is_binary() - || params.group_data_types[0].is_variant() - { - ( - Some(BinaryColumnBuilder::with_capacity(capacity, data_capacity)), - None, - vec![], - ) - } else { - (None, None, Vec::with_capacity(capacity)) - } - } else { - (None, None, Vec::with_capacity(capacity)) - }; - - Self { - data, - group_data_types: params.group_data_types.clone(), - single_binary_builder, - single_string_builder, - } - } -} - -impl<'a> GroupColumnsBuilder for SerializedKeysGroupColumnsBuilder<'a> { - type T = &'a [u8]; - - fn append_value(&mut self, v: &'a [u8]) { - match ( - self.single_string_builder.as_mut(), - self.single_binary_builder.as_mut(), - ) { - (Some(s), _) => { - s.put_slice(v); - s.commit_row(); - } - (_, Some(s)) => { - s.put_slice(v); - s.commit_row(); - } - (_, _) => self.data.push(v), - } - } - - fn finish(mut self) -> Result> { - if let Some(builder) = self.single_binary_builder.take() { - let col = builder.build(); - match self.group_data_types[0] { - DataType::Binary => return Ok(vec![Column::Binary(col)]), - DataType::Variant => return Ok(vec![Column::Variant(col)]), - _ => {} - } - } else if let Some(builder) = self.single_string_builder.take() { - let col = builder.build(); - return Ok(vec![Column::String(col)]); - } - - let rows = self.data.len(); - let keys = self.data.as_mut_slice(); - - let mut res = Vec::with_capacity(self.group_data_types.len()); - for data_type in self.group_data_types.iter() { - let mut column = ColumnBuilder::with_capacity(data_type, rows); - - for key in keys.iter_mut() { - column.push_binary(key)?; - } - res.push(column.build()); - } - - Ok(res) - } -} - -pub struct DictionarySerializedKeysGroupColumnsBuilder<'a> { - other_type_data: Vec<&'a [u8]>, - string_type_data: Vec, - group_data_types: Vec, -} - -impl DictionarySerializedKeysGroupColumnsBuilder<'_> { - pub fn create(capacity: usize, _data_capacity: usize, params: &AggregatorParams) -> Self { - Self { - other_type_data: Vec::with_capacity(capacity), - string_type_data: Vec::with_capacity(capacity), - group_data_types: params.group_data_types.clone(), - } - } -} - -impl<'a> GroupColumnsBuilder for DictionarySerializedKeysGroupColumnsBuilder<'a> { - type T = &'a DictionaryKeys; - - fn append_value(&mut self, v: &'a DictionaryKeys) { - unsafe { - if let Some(last) = v.keys.as_ref().last() { - self.other_type_data.push(last.as_ref()); - } - } - - self.string_type_data.push(*v) - } - - fn finish(mut self) -> Result> { - let rows = self.string_type_data.len(); - let other_type_keys = self.other_type_data.as_mut_slice(); - - let mut index = 0; - let mut res = Vec::with_capacity(self.group_data_types.len()); - for data_type in self.group_data_types.iter() { - if data_type.is_variant() { - let mut builder = BinaryColumnBuilder::with_capacity(0, 0); - - for string_type_keys in &self.string_type_data { - builder.put(unsafe { string_type_keys.keys.as_ref()[index].as_ref() }); - builder.commit_row(); - } - - index += 1; - res.push(Column::Variant(builder.build())); - } else { - let mut column = ColumnBuilder::with_capacity(data_type, rows); - - for key in other_type_keys.iter_mut() { - column.push_binary(key)?; - } - - res.push(column.build()); - } - } - - Ok(res) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs deleted file mode 100644 index 11dfdcffa818..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_builder.rs +++ /dev/null @@ -1,187 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::marker::PhantomData; - -use byteorder::BigEndian; -use byteorder::WriteBytesExt; -use databend_common_column::buffer::Buffer; -use databend_common_expression::types::binary::BinaryColumnBuilder; -use databend_common_expression::types::decimal::Decimal; -use databend_common_expression::types::number::Number; -use databend_common_expression::types::NumberType; -use databend_common_expression::types::ValueType; -use databend_common_expression::Column; -use databend_common_hashtable::DictionaryKeys; -use ethnum::i256; - -use super::large_number::LargeNumber; - -/// Remove the group by key from the state and rebuild it into a column -pub trait KeysColumnBuilder { - type T; - - fn bytes_size(&self) -> usize; - fn append_value(&mut self, v: Self::T); - fn finish(self) -> Column; -} - -pub struct FixedKeysColumnBuilder<'a, T: Number> { - pub _t: PhantomData<&'a ()>, - pub inner_builder: Vec, -} - -impl<'a, T: Number> KeysColumnBuilder for FixedKeysColumnBuilder<'a, T> { - type T = &'a T; - - fn bytes_size(&self) -> usize { - self.inner_builder.len() * std::mem::size_of::() - } - - #[inline] - fn append_value(&mut self, v: Self::T) { - self.inner_builder.push(*v) - } - - #[inline] - fn finish(self) -> Column { - NumberType::::upcast_column(NumberType::::build_column(self.inner_builder)) - } -} - -pub struct BinaryKeysColumnBuilder<'a> { - pub inner_builder: BinaryColumnBuilder, - - _initial: usize, - - _phantom: PhantomData<&'a ()>, -} - -impl BinaryKeysColumnBuilder<'_> { - pub fn create(capacity: usize, value_capacity: usize) -> Self { - BinaryKeysColumnBuilder { - inner_builder: BinaryColumnBuilder::with_capacity(capacity, value_capacity), - _phantom: PhantomData, - _initial: value_capacity, - } - } -} - -impl<'a> KeysColumnBuilder for BinaryKeysColumnBuilder<'a> { - type T = &'a [u8]; - - fn bytes_size(&self) -> usize { - self.inner_builder.data.len() - } - - fn append_value(&mut self, v: &'a [u8]) { - self.inner_builder.put_slice(v); - self.inner_builder.commit_row(); - } - - fn finish(self) -> Column { - Column::Binary(self.inner_builder.build()) - } -} - -pub struct LargeFixedKeysColumnBuilder<'a, T: LargeNumber> { - pub _t: PhantomData<&'a ()>, - pub values: Vec, -} - -impl<'a, T: LargeNumber> KeysColumnBuilder for LargeFixedKeysColumnBuilder<'a, T> { - type T = &'a T; - - fn bytes_size(&self) -> usize { - self.values.len() * std::mem::size_of::() - } - - #[inline] - fn append_value(&mut self, v: Self::T) { - self.values.push(*v); - } - - #[inline] - fn finish(self) -> Column { - match T::BYTE_SIZE { - 16 => { - let values: Buffer = self.values.into(); - let values: Buffer = unsafe { std::mem::transmute(values) }; - let col = i128::to_column_from_buffer(values, i128::default_decimal_size()); - Column::Decimal(col) - } - 32 => { - let values: Buffer = self.values.into(); - let values: Buffer = unsafe { std::mem::transmute(values) }; - let col = i256::to_column_from_buffer(values, i256::default_decimal_size()); - Column::Decimal(col) - } - _ => unreachable!(), - } - } -} - -pub struct DictionaryBinaryKeysColumnBuilder<'a> { - bytes_size: usize, - data: Vec, - _phantom: PhantomData<&'a ()>, -} - -impl DictionaryBinaryKeysColumnBuilder<'_> { - pub fn create(_: usize, _: usize) -> Self { - DictionaryBinaryKeysColumnBuilder { - bytes_size: 0, - data: vec![], - _phantom: PhantomData, - } - } -} - -impl<'a> KeysColumnBuilder for DictionaryBinaryKeysColumnBuilder<'a> { - type T = &'a DictionaryKeys; - - fn bytes_size(&self) -> usize { - self.bytes_size - } - - #[inline(always)] - fn append_value(&mut self, v: &'a DictionaryKeys) { - unsafe { - for x in v.keys.as_ref() { - self.bytes_size += x.as_ref().len() + std::mem::size_of::(); - } - } - - self.data.push(*v) - } - - #[inline(always)] - fn finish(self) -> Column { - let mut builder = BinaryColumnBuilder::with_capacity(self.data.len(), self.bytes_size); - - unsafe { - for dictionary_keys in self.data { - for dictionary_key in dictionary_keys.keys.as_ref() { - let i = dictionary_key.as_ref().len() as u64; - builder.data.write_u64::(i).unwrap(); - builder.put(dictionary_key.as_ref()); - } - - builder.commit_row(); - } - } - - Column::Binary(builder.build()) - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs deleted file mode 100644 index 590f1f1a2548..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_keys_iter.rs +++ /dev/null @@ -1,145 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::ptr::NonNull; -use std::slice::Iter; - -use byteorder::BigEndian; -use byteorder::ReadBytesExt; -use databend_common_column::buffer::Buffer; -use databend_common_exception::Result; -use databend_common_expression::types::binary::BinaryColumn; -use databend_common_expression::types::binary::BinaryColumnIter; -use databend_common_expression::types::number::Number; -use databend_common_hashtable::DictionaryKeys; - -use super::large_number::LargeNumber; - -pub trait KeysColumnIter { - type Iterator<'a>: Iterator - where - Self: 'a, - T: 'a; - - fn iter(&self) -> Self::Iterator<'_>; -} - -pub struct FixedKeysColumnIter { - column: Buffer, -} - -impl FixedKeysColumnIter { - pub fn create(column: &Buffer) -> Result { - Ok(Self { - column: column.clone(), - }) - } -} - -impl KeysColumnIter for FixedKeysColumnIter { - type Iterator<'a> - = Iter<'a, T> - where - Self: 'a, - T: 'a; - - fn iter(&self) -> Self::Iterator<'_> { - self.column.iter() - } -} - -pub struct LargeFixedKeysColumnIter { - holder: Buffer, -} - -impl LargeFixedKeysColumnIter { - pub fn create(holder: Buffer) -> Result { - Ok(Self { holder }) - } -} - -impl KeysColumnIter for LargeFixedKeysColumnIter { - type Iterator<'a> - = Iter<'a, T> - where - Self: 'a, - T: 'a; - - fn iter(&self) -> Self::Iterator<'_> { - self.holder.iter() - } -} - -pub struct SerializedKeysColumnIter { - column: BinaryColumn, -} - -impl SerializedKeysColumnIter { - pub fn create(column: &BinaryColumn) -> Result { - Ok(SerializedKeysColumnIter { - column: column.clone(), - }) - } -} - -impl KeysColumnIter<[u8]> for SerializedKeysColumnIter { - type Iterator<'a> - = BinaryColumnIter<'a> - where Self: 'a; - - fn iter(&self) -> Self::Iterator<'_> { - self.column.iter() - } -} - -pub struct DictionarySerializedKeysColumnIter { - #[allow(dead_code)] - column: BinaryColumn, - #[allow(dead_code)] - points: Vec>, - inner: Vec, -} - -impl DictionarySerializedKeysColumnIter { - pub fn create( - dict_keys: usize, - column: &BinaryColumn, - ) -> Result { - let mut inner = Vec::with_capacity(column.len()); - let mut points = Vec::with_capacity(column.len() * dict_keys); - for (index, mut item) in column.iter().enumerate() { - for _ in 0..dict_keys { - let len = item.read_u64::()? as usize; - points.push(NonNull::from(&item[0..len])); - item = &item[len..]; - } - - inner.push(DictionaryKeys::create(&points[index * dict_keys..])); - } - - Ok(DictionarySerializedKeysColumnIter { - inner, - points, - column: column.clone(), - }) - } -} - -impl KeysColumnIter for DictionarySerializedKeysColumnIter { - type Iterator<'a> = std::slice::Iter<'a, DictionaryKeys>; - - fn iter(&self) -> Self::Iterator<'_> { - self.inner.iter() - } -} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs deleted file mode 100644 index 3704c0eb0748..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_polymorphic_keys.rs +++ /dev/null @@ -1,702 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::marker::PhantomData; -use std::sync::Arc; -use std::time::Instant; - -use bumpalo::Bump; -use databend_common_exception::ErrorCode; -use databend_common_exception::Result; -use databend_common_expression::types::number::*; -use databend_common_expression::types::ValueType; -use databend_common_expression::Column; -use databend_common_expression::HashMethod; -use databend_common_expression::HashMethodDictionarySerializer; -use databend_common_expression::HashMethodFixedKeys; -use databend_common_expression::HashMethodKeysU128; -use databend_common_expression::HashMethodKeysU256; -use databend_common_expression::HashMethodSerializer; -use databend_common_expression::HashMethodSingleBinary; -use databend_common_expression::InputColumns; -use databend_common_expression::KeyAccessor; -use databend_common_expression::KeysState; -use databend_common_hashtable::DictionaryKeys; -use databend_common_hashtable::DictionaryStringHashMap; -use databend_common_hashtable::FastHash; -use databend_common_hashtable::HashMap; -use databend_common_hashtable::HashtableEntryMutRefLike; -use databend_common_hashtable::HashtableEntryRefLike; -use databend_common_hashtable::HashtableLike; -use databend_common_hashtable::LookupHashMap; -use databend_common_hashtable::PartitionedHashMap; -use databend_common_hashtable::ShortStringHashMap; -use databend_common_hashtable::StringHashMap; -use ethnum::U256; -use log::info; - -use super::aggregator_keys_builder::LargeFixedKeysColumnBuilder; -use super::aggregator_keys_iter::LargeFixedKeysColumnIter; -use super::BUCKETS_LG2; -use crate::pipelines::processors::transforms::aggregator::AggregatorParams; -use crate::pipelines::processors::transforms::aggregator::HashTableCell; -use crate::pipelines::processors::transforms::aggregator::PartitionedHashTableDropper; -use crate::pipelines::processors::transforms::group_by::Area; -use crate::pipelines::processors::transforms::group_by::ArenaHolder; -use crate::pipelines::processors::transforms::group_by::BinaryKeysColumnBuilder; -use crate::pipelines::processors::transforms::group_by::DictionaryBinaryKeysColumnBuilder; -use crate::pipelines::processors::transforms::group_by::DictionarySerializedKeysColumnIter; -use crate::pipelines::processors::transforms::group_by::DictionarySerializedKeysGroupColumnsBuilder; -use crate::pipelines::processors::transforms::group_by::FixedKeysColumnBuilder; -use crate::pipelines::processors::transforms::group_by::FixedKeysColumnIter; -use crate::pipelines::processors::transforms::group_by::FixedKeysGroupColumnsBuilder; -use crate::pipelines::processors::transforms::group_by::GroupColumnsBuilder; -use crate::pipelines::processors::transforms::group_by::KeysColumnBuilder; -use crate::pipelines::processors::transforms::group_by::KeysColumnIter; -use crate::pipelines::processors::transforms::group_by::SerializedKeysColumnIter; -use crate::pipelines::processors::transforms::group_by::SerializedKeysGroupColumnsBuilder; - -// Provide functions for all HashMethod to help implement polymorphic group by key -// -// When we want to add new HashMethod, we need to add the following components -// - HashMethod, more information in [HashMethod] trait -// - AggregatorState, more information in [AggregatorState] trait -// - KeysColumnBuilder, more information in [KeysColumnBuilder] trait -// - PolymorphicKeysHelper, more information in following comments -// -// For example: -// -// use bumpalo::Bump; -// use databend_query::common::HashTable; -// use databend_common_expression::HashMethodSerializer; -// use databend_query::pipelines::processors::transforms::group_by::PolymorphicKeysHelper; -// use databend_query::pipelines::processors::transforms::group_by::aggregator_state::SerializedKeysAggregatorState; -// use databend_query::pipelines::processors::transforms::group_by::aggregator_keys_builder::StringKeysColumnBuilder; -// -// impl PolymorphicKeysHelper for HashMethodSerializer { -// type State = SerializedKeysAggregatorState; -// fn aggregate_state(&self) -> Self::State { -// SerializedKeysAggregatorState { -// keys_area: Bump::new(), -// state_area: Bump::new(), -// data_state_map: HashTable::create(), -// } -// } -// -// type ColumnBuilder = StringKeysColumnBuilder; -// fn state_array_builder(&self, capacity: usize) -> Self::ColumnBuilder { -// StringKeysColumnBuilder { -// inner_builder: MutableStringColumn::with_capacity(capacity), -// } -// } -// } -// -pub trait PolymorphicKeysHelper: Send + Sync + 'static { - const SUPPORT_PARTITIONED: bool; - - type HashTable: HashtableLike - + Send - + Sync - + 'static; - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result>; - - type ColumnBuilder<'a>: KeysColumnBuilder - where - Self: 'a, - Method: 'a; - - fn keys_column_builder( - &self, - capacity: usize, - value_capacity: usize, - ) -> Self::ColumnBuilder<'_>; - - type KeysColumnIter: KeysColumnIter; - - fn keys_iter_from_column(&self, column: &Column) -> Result; - - type GroupColumnsBuilder<'a>: GroupColumnsBuilder - where - Self: 'a, - Method: 'a; - - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> Self::GroupColumnsBuilder<'_>; - - fn get_hash(&self, v: &Method::HashKey) -> u64; -} - -impl PolymorphicKeysHelper> for HashMethodFixedKeys { - const SUPPORT_PARTITIONED: bool = false; - - type HashTable = LookupHashMap; - - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result> { - Ok(LookupHashMap::create(Default::default())) - } - - type ColumnBuilder<'a> = FixedKeysColumnBuilder<'a, u8>; - fn keys_column_builder(&self, capacity: usize, _: usize) -> FixedKeysColumnBuilder { - FixedKeysColumnBuilder:: { - _t: Default::default(), - inner_builder: Vec::with_capacity(capacity), - } - } - type KeysColumnIter = FixedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - FixedKeysColumnIter::create(&UInt8Type::try_downcast_column(column).ok_or_else(|| { - ErrorCode::IllegalDataType("Illegal data type for FixedKeysColumnIter".to_string()) - })?) - } - type GroupColumnsBuilder<'a> = FixedKeysGroupColumnsBuilder<'a, u8>; - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> FixedKeysGroupColumnsBuilder { - FixedKeysGroupColumnsBuilder::::create(capacity, params) - } - - fn get_hash(&self, v: &u8) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper> for HashMethodFixedKeys { - const SUPPORT_PARTITIONED: bool = false; - - type HashTable = LookupHashMap; - - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result> { - Ok(LookupHashMap::create(Default::default())) - } - - type ColumnBuilder<'a> = FixedKeysColumnBuilder<'a, u16>; - fn keys_column_builder(&self, capacity: usize, _: usize) -> FixedKeysColumnBuilder { - FixedKeysColumnBuilder:: { - _t: Default::default(), - inner_builder: Vec::with_capacity(capacity), - } - } - type KeysColumnIter = FixedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - FixedKeysColumnIter::create(&UInt16Type::try_downcast_column(column).ok_or_else(|| { - ErrorCode::IllegalDataType("Illegal data type for FixedKeysColumnIter".to_string()) - })?) - } - type GroupColumnsBuilder<'a> = FixedKeysGroupColumnsBuilder<'a, u16>; - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> FixedKeysGroupColumnsBuilder { - FixedKeysGroupColumnsBuilder::::create(capacity, params) - } - - fn get_hash(&self, v: &u16) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper> for HashMethodFixedKeys { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = HashMap; - - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result> { - Ok(HashMap::new()) - } - - type ColumnBuilder<'a> = FixedKeysColumnBuilder<'a, u32>; - fn keys_column_builder(&self, capacity: usize, _: usize) -> FixedKeysColumnBuilder { - FixedKeysColumnBuilder:: { - _t: Default::default(), - inner_builder: Vec::with_capacity(capacity), - } - } - type KeysColumnIter = FixedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - FixedKeysColumnIter::create(&UInt32Type::try_downcast_column(column).ok_or_else(|| { - ErrorCode::IllegalDataType("Illegal data type for FixedKeysColumnIter".to_string()) - })?) - } - type GroupColumnsBuilder<'a> = FixedKeysGroupColumnsBuilder<'a, u32>; - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> FixedKeysGroupColumnsBuilder { - FixedKeysGroupColumnsBuilder::::create(capacity, params) - } - - fn get_hash(&self, v: &u32) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper> for HashMethodFixedKeys { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = HashMap; - - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result> { - Ok(HashMap::new()) - } - - type ColumnBuilder<'a> = FixedKeysColumnBuilder<'a, u64>; - fn keys_column_builder(&self, capacity: usize, _: usize) -> FixedKeysColumnBuilder { - FixedKeysColumnBuilder:: { - _t: Default::default(), - inner_builder: Vec::with_capacity(capacity), - } - } - type KeysColumnIter = FixedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - FixedKeysColumnIter::create(&UInt64Type::try_downcast_column(column).ok_or_else(|| { - ErrorCode::IllegalDataType("Illegal data type for FixedKeysColumnIter".to_string()) - })?) - } - type GroupColumnsBuilder<'a> = FixedKeysGroupColumnsBuilder<'a, u64>; - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> FixedKeysGroupColumnsBuilder { - FixedKeysGroupColumnsBuilder::::create(capacity, params) - } - - fn get_hash(&self, v: &u64) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper for HashMethodKeysU128 { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = HashMap; - - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result> { - Ok(HashMap::new()) - } - - type ColumnBuilder<'a> = LargeFixedKeysColumnBuilder<'a, u128>; - fn keys_column_builder(&self, capacity: usize, _: usize) -> LargeFixedKeysColumnBuilder { - LargeFixedKeysColumnBuilder:: { - values: Vec::with_capacity(capacity * 16), - _t: PhantomData, - } - } - - type KeysColumnIter = LargeFixedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - let buffer = column - .as_decimal() - .and_then(|c| c.as_decimal128()) - .ok_or_else(|| { - ErrorCode::IllegalDataType( - "Illegal data type for LargeFixedKeysColumnIter".to_string(), - ) - })?; - let buffer = unsafe { std::mem::transmute::, Buffer>(buffer.0.clone()) }; - LargeFixedKeysColumnIter::create(buffer) - } - - type GroupColumnsBuilder<'a> = FixedKeysGroupColumnsBuilder<'a, u128>; - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> FixedKeysGroupColumnsBuilder { - FixedKeysGroupColumnsBuilder::create(capacity, params) - } - - fn get_hash(&self, v: &u128) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper for HashMethodKeysU256 { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = HashMap; - - fn create_hash_table( - &self, - _bump: Arc, - ) -> Result> { - Ok(HashMap::new()) - } - - type ColumnBuilder<'a> = LargeFixedKeysColumnBuilder<'a, U256>; - fn keys_column_builder(&self, capacity: usize, _: usize) -> LargeFixedKeysColumnBuilder { - LargeFixedKeysColumnBuilder:: { - values: Vec::with_capacity(capacity * 32), - _t: PhantomData, - } - } - - type KeysColumnIter = LargeFixedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - let buffer = column - .as_decimal() - .and_then(|c| c.as_decimal256()) - .ok_or_else(|| { - ErrorCode::IllegalDataType( - "Illegal data type for LargeFixedKeysColumnIter".to_string(), - ) - })?; - let buffer = unsafe { - std::mem::transmute::, Buffer>(buffer.0.clone()) - }; - - LargeFixedKeysColumnIter::create(buffer) - } - - type GroupColumnsBuilder<'a> = FixedKeysGroupColumnsBuilder<'a, U256>; - fn group_columns_builder( - &self, - capacity: usize, - _data_capacity: usize, - params: &AggregatorParams, - ) -> FixedKeysGroupColumnsBuilder { - FixedKeysGroupColumnsBuilder::create(capacity, params) - } - - fn get_hash(&self, v: &U256) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper for HashMethodSingleBinary { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = ShortStringHashMap<[u8], T>; - - fn create_hash_table( - &self, - bump: Arc, - ) -> Result> { - Ok(ShortStringHashMap::new(bump)) - } - - type ColumnBuilder<'a> = BinaryKeysColumnBuilder<'a>; - fn keys_column_builder( - &self, - capacity: usize, - value_capacity: usize, - ) -> BinaryKeysColumnBuilder<'_> { - BinaryKeysColumnBuilder::create(capacity, value_capacity) - } - - type KeysColumnIter = SerializedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - SerializedKeysColumnIter::create(column.as_binary().ok_or_else(|| { - ErrorCode::IllegalDataType("Illegal data type for SerializedKeysColumnIter".to_string()) - })?) - } - - type GroupColumnsBuilder<'a> = SerializedKeysGroupColumnsBuilder<'a>; - fn group_columns_builder( - &self, - capacity: usize, - data_capacity: usize, - params: &AggregatorParams, - ) -> SerializedKeysGroupColumnsBuilder<'_> { - SerializedKeysGroupColumnsBuilder::create(capacity, data_capacity, params) - } - - fn get_hash(&self, v: &[u8]) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper for HashMethodSerializer { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = StringHashMap<[u8], T>; - - fn create_hash_table( - &self, - bump: Arc, - ) -> Result> { - Ok(StringHashMap::new(bump)) - } - - type ColumnBuilder<'a> = BinaryKeysColumnBuilder<'a>; - fn keys_column_builder( - &self, - capacity: usize, - value_capacity: usize, - ) -> BinaryKeysColumnBuilder<'_> { - BinaryKeysColumnBuilder::create(capacity, value_capacity) - } - - type KeysColumnIter = SerializedKeysColumnIter; - fn keys_iter_from_column(&self, column: &Column) -> Result { - SerializedKeysColumnIter::create(column.as_binary().ok_or_else(|| { - ErrorCode::IllegalDataType("Illegal data type for SerializedKeysColumnIter".to_string()) - })?) - } - - type GroupColumnsBuilder<'a> = SerializedKeysGroupColumnsBuilder<'a>; - fn group_columns_builder( - &self, - capacity: usize, - data_capacity: usize, - params: &AggregatorParams, - ) -> SerializedKeysGroupColumnsBuilder<'_> { - SerializedKeysGroupColumnsBuilder::create(capacity, data_capacity, params) - } - - fn get_hash(&self, v: &[u8]) -> u64 { - v.fast_hash() - } -} - -impl PolymorphicKeysHelper for HashMethodDictionarySerializer { - const SUPPORT_PARTITIONED: bool = true; - - type HashTable = DictionaryStringHashMap; - - fn create_hash_table( - &self, - bump: Arc, - ) -> Result> { - Ok(DictionaryStringHashMap::new(bump, self.dict_keys)) - } - - type ColumnBuilder<'a> = DictionaryBinaryKeysColumnBuilder<'a>; - - fn keys_column_builder( - &self, - capacity: usize, - value_capacity: usize, - ) -> Self::ColumnBuilder<'_> { - DictionaryBinaryKeysColumnBuilder::create(capacity, value_capacity) - } - - type KeysColumnIter = DictionarySerializedKeysColumnIter; - - fn keys_iter_from_column(&self, column: &Column) -> Result { - DictionarySerializedKeysColumnIter::create( - self.dict_keys, - column.as_binary().ok_or_else(|| { - ErrorCode::IllegalDataType( - "Illegal data type for SerializedKeysColumnIter".to_string(), - ) - })?, - ) - } - - type GroupColumnsBuilder<'a> = DictionarySerializedKeysGroupColumnsBuilder<'a>; - - fn group_columns_builder( - &self, - capacity: usize, - data_capacity: usize, - params: &AggregatorParams, - ) -> Self::GroupColumnsBuilder<'_> { - DictionarySerializedKeysGroupColumnsBuilder::create(capacity, data_capacity, params) - } - - fn get_hash(&self, v: &DictionaryKeys) -> u64 { - v.fast_hash() - } -} - -#[derive(Clone)] -pub struct PartitionedHashMethod { - pub(crate) method: Method, -} - -impl PartitionedHashMethod { - pub fn create(method: Method) -> PartitionedHashMethod { - PartitionedHashMethod:: { method } - } - - pub fn convert_hashtable( - method: &Method, - mut cell: HashTableCell, - ) -> Result, T>> - where - T: Copy + Send + Sync + 'static, - Self: PolymorphicKeysHelper>, - { - let instant = Instant::now(); - let arena = Arc::new(Bump::new()); - let partitioned_method = Self::create(method.clone()); - let mut partitioned_hashtable = partitioned_method.create_hash_table(arena)?; - - unsafe { - for item in cell.hashtable.iter() { - match partitioned_hashtable.insert_and_entry(item.key()) { - Ok(mut entry) => { - *entry.get_mut() = *item.get(); - } - Err(mut entry) => { - *entry.get_mut() = *item.get(); - } - }; - } - } - - info!( - "Convert to Partitioned HashTable elapsed: {:?}", - instant.elapsed() - ); - - let arena = std::mem::replace(&mut cell.arena, Area::create()); - cell.arena_holders.push(ArenaHolder::create(Some(arena))); - let arena_holders = cell.arena_holders.to_vec(); - - let _old_dropper = cell._dropper.clone().unwrap(); - let _new_dropper = PartitionedHashTableDropper::::create(_old_dropper); - - // TODO(winter): No idea(may memory leak). - // We need to ensure that the following two lines of code are atomic. - // take_old_dropper before create new HashTableCell - may memory leak - // create new HashTableCell before take_old_dropper - may double free memory - let _old_dropper = cell._dropper.take(); - let mut cell = HashTableCell::create(partitioned_hashtable, _new_dropper); - cell.arena_holders = arena_holders; - Ok(cell) - } -} - -impl HashMethod for PartitionedHashMethod { - type HashKey = Method::HashKey; - type HashKeyIter<'a> - = Method::HashKeyIter<'a> - where Self: 'a; - - fn name(&self) -> String { - format!("Partitioned{}", self.method.name()) - } - - fn build_keys_state(&self, group_columns: InputColumns, rows: usize) -> Result { - self.method.build_keys_state(group_columns, rows) - } - - fn build_keys_iter<'a>(&self, keys_state: &'a KeysState) -> Result> { - self.method.build_keys_iter(keys_state) - } - - fn build_keys_accessor( - &self, - keys_state: KeysState, - ) -> Result>> { - self.method.build_keys_accessor(keys_state) - } - - fn build_keys_hashes(&self, keys_state: &KeysState, hashes: &mut Vec) { - self.method.build_keys_hashes(keys_state, hashes) - } -} - -impl PolymorphicKeysHelper> for PartitionedHashMethod -where - Self: HashMethod, - Method: HashMethod + PolymorphicKeysHelper, -{ - // Partitioned cannot be recursive - const SUPPORT_PARTITIONED: bool = false; - - type HashTable = - PartitionedHashMap, BUCKETS_LG2>; - - fn create_hash_table( - &self, - arena: Arc, - ) -> Result> { - let buckets = (1 << BUCKETS_LG2) as usize; - let mut tables = Vec::with_capacity(buckets); - - for _index in 0..buckets { - tables.push(self.method.create_hash_table(arena.clone())?); - } - - Ok(PartitionedHashMap::<_, BUCKETS_LG2>::create(arena, tables)) - } - - type ColumnBuilder<'a> - = Method::ColumnBuilder<'a> - where - Self: 'a, - PartitionedHashMethod: 'a; - - fn keys_column_builder( - &self, - capacity: usize, - value_capacity: usize, - ) -> Self::ColumnBuilder<'_> { - self.method.keys_column_builder(capacity, value_capacity) - } - - type KeysColumnIter = Method::KeysColumnIter; - - fn keys_iter_from_column(&self, column: &Column) -> Result { - self.method.keys_iter_from_column(column) - } - - type GroupColumnsBuilder<'a> - = Method::GroupColumnsBuilder<'a> - where - Self: 'a, - PartitionedHashMethod: 'a; - - fn group_columns_builder( - &self, - capacity: usize, - data_capacity: usize, - params: &AggregatorParams, - ) -> Self::GroupColumnsBuilder<'_> { - self.method - .group_columns_builder(capacity, data_capacity, params) - } - - fn get_hash(&self, v: &Method::HashKey) -> u64 { - self.method.get_hash(v) - } -} - -pub trait HashMethodBounds: HashMethod + PolymorphicKeysHelper {} - -impl> HashMethodBounds for T {} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_state.rs b/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_state.rs deleted file mode 100644 index 67386e1ea40a..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/aggregator_state.rs +++ /dev/null @@ -1,70 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use std::alloc::Layout; -use std::fmt::Debug; -use std::ptr::NonNull; -use std::sync::Arc; - -use bumpalo::Bump; - -pub struct Area { - bump: Bump, -} - -impl Area { - pub fn create() -> Area { - Area { bump: Bump::new() } - } - - pub fn allocated_bytes(&self) -> usize { - self.bump.allocated_bytes() - } - - pub fn alloc_layout(&mut self, layout: Layout) -> NonNull { - self.bump.alloc_layout(layout) - } -} - -unsafe impl Send for Area {} - -#[derive(Clone)] -pub struct ArenaHolder { - _data: Arc>, -} - -impl ArenaHolder { - pub fn create(area: Option) -> ArenaHolder { - ArenaHolder { - _data: Arc::new(area), - } - } - - pub fn allocated_bytes(&self) -> usize { - match self._data.as_ref() { - None => 0, - Some(arena) => arena.allocated_bytes(), - } - } -} - -impl Debug for ArenaHolder { - fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { - f.debug_struct("ArenaHolder").finish() - } -} - -unsafe impl Send for ArenaHolder {} - -unsafe impl Sync for ArenaHolder {} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/large_number.rs b/src/query/service/src/pipelines/processors/transforms/group_by/large_number.rs deleted file mode 100644 index 50c2608351e7..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/large_number.rs +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -use ethnum::U256; - -pub trait LargeNumber: Default + Copy + Sized + 'static { - const BYTE_SIZE: usize; -} - -impl LargeNumber for u128 { - const BYTE_SIZE: usize = 16; -} - -impl LargeNumber for U256 { - const BYTE_SIZE: usize = 32; -} diff --git a/src/query/service/src/pipelines/processors/transforms/group_by/mod.rs b/src/query/service/src/pipelines/processors/transforms/group_by/mod.rs deleted file mode 100644 index 15ec8fc2ac02..000000000000 --- a/src/query/service/src/pipelines/processors/transforms/group_by/mod.rs +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright 2021 Datafuse Labs -// -// Licensed 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. - -mod aggregator_groups_builder; -mod aggregator_keys_builder; -mod aggregator_keys_iter; -mod aggregator_polymorphic_keys; -mod aggregator_state; -mod large_number; - -pub use aggregator_groups_builder::*; -pub use aggregator_keys_builder::*; -pub use aggregator_keys_iter::*; -pub use aggregator_polymorphic_keys::*; -pub use aggregator_state::*; - -pub const BUCKETS_LG2: u32 = 8; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs index 440178b6268b..e110681f17d4 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_build_state.rs @@ -143,7 +143,7 @@ impl HashJoinBuildState { .remove_nullable() }) .collect::>(); - let method = DataBlock::choose_hash_method_with_types(&hash_key_types, false)?; + let method = DataBlock::choose_hash_method_with_types(&hash_key_types)?; let mut enable_bloom_runtime_filter = false; let mut enable_inlist_runtime_filter = false; let mut enable_min_max_runtime_filter = false; @@ -415,7 +415,6 @@ impl HashJoinBuildState { }), std::mem::size_of::>(), ), - HashMethodKind::DictionarySerializer(_) => unimplemented!(), }; self.entry_size.store(entry_size, Ordering::Release); let hash_table = unsafe { &mut *self.hash_join_state.hash_table.get() }; @@ -967,7 +966,7 @@ impl HashJoinBuildState { // Generate bloom filter using build column let data_type = build_key.data_type(); let num_rows = build_key_column.len(); - let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()], false)?; + let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()])?; let mut hashes = HashSet::with_capacity(num_rows); let key_columns = &[build_key_column]; hash_by_method(&method, key_columns.into(), num_rows, &mut hashes)?; diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs index 5cb8b8e835de..548a6062d00d 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/hash_join_probe_state.rs @@ -121,7 +121,7 @@ impl HashJoinProbeState { .clone() }) .collect::>(); - let method = DataBlock::choose_hash_method_with_types(&hash_key_types, false)?; + let method = DataBlock::choose_hash_method_with_types(&hash_key_types)?; Ok(HashJoinProbeState { ctx, func_ctx, diff --git a/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs b/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs index fb24d7f951a3..4f9b5335f87b 100644 --- a/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs +++ b/src/query/service/src/pipelines/processors/transforms/hash_join/util.rs @@ -163,14 +163,6 @@ where .map(|key| key.fast_hash()), ); } - HashMethodKind::DictionarySerializer(method) => { - let keys_state = method.build_keys_state(columns, num_rows)?; - hashes.extend( - method - .build_keys_iter(&keys_state)? - .map(|key| key.fast_hash()), - ); - } HashMethodKind::SingleBinary(method) => { let keys_state = method.build_keys_state(columns, num_rows)?; hashes.extend( diff --git a/src/query/service/src/pipelines/processors/transforms/mod.rs b/src/query/service/src/pipelines/processors/transforms/mod.rs index 706135c9b079..e4d503c0f37f 100644 --- a/src/query/service/src/pipelines/processors/transforms/mod.rs +++ b/src/query/service/src/pipelines/processors/transforms/mod.rs @@ -13,7 +13,6 @@ // limitations under the License. pub mod aggregator; -pub mod group_by; mod hash_join; pub(crate) mod range_join; mod transform_add_computed_columns; diff --git a/src/query/service/src/pipelines/processors/transforms/window/window_function.rs b/src/query/service/src/pipelines/processors/transforms/window/window_function.rs index 7cdaf08aad78..f2feb1690758 100644 --- a/src/query/service/src/pipelines/processors/transforms/window/window_function.rs +++ b/src/query/service/src/pipelines/processors/transforms/window/window_function.rs @@ -29,8 +29,6 @@ use databend_common_functions::aggregates::StateAddr; use databend_common_sql::executor::physical_plans::LagLeadDefault; use databend_common_sql::executor::physical_plans::WindowFunction; -use crate::pipelines::processors::transforms::group_by::Area; - #[derive(Clone)] pub enum WindowFunctionInfo { // (func instance, argument offsets) @@ -45,9 +43,10 @@ pub enum WindowFunctionInfo { CumeDist, } +type Arena = bumpalo::Bump; pub struct WindowFuncAggImpl { // Need to hold arena until `drop`. - _arena: Area, + _arena: Arena, agg: Arc, place: StateAddr, args: Vec, @@ -233,7 +232,7 @@ impl WindowFunctionImpl { pub(crate) fn try_create(window: WindowFunctionInfo) -> Result { Ok(match window { WindowFunctionInfo::Aggregate(agg, args) => { - let mut arena = Area::create(); + let arena = Arena::new(); let mut state_offset = Vec::with_capacity(1); let layout = get_layout_offsets(&[agg.clone()], &mut state_offset)?; let place: StateAddr = arena.alloc_layout(layout).into(); diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs index b7a7197287af..fdab0c47810c 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_final.rs @@ -16,7 +16,6 @@ use std::sync::Arc; use databend_common_exception::ErrorCode; use databend_common_exception::Result; -use databend_common_expression::DataBlock; use databend_common_expression::DataField; use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; @@ -224,12 +223,7 @@ impl PhysicalPlanBuilder { } }; - let settings = self.ctx.get_settings(); - let efficiently_memory = settings.get_efficiently_memory_group_by()?; - let enable_experimental_aggregate_hashtable = - settings.get_enable_experimental_aggregate_hashtable()?; - - let keys = if enable_experimental_aggregate_hashtable { + let keys = { let schema = aggregate_partial.output_schema()?; let start = aggregate_partial.agg_funcs.len(); let end = schema.num_fields(); @@ -244,23 +238,6 @@ impl PhysicalPlanBuilder { groups.push(group_key); } groups - } else { - let group_by_key_index = - aggregate_partial.output_schema()?.num_fields() - 1; - let group_by_key_data_type = DataBlock::choose_hash_method_with_types( - &agg.group_items - .iter() - .map(|v| v.scalar.data_type()) - .collect::>>()?, - efficiently_memory, - )? - .data_type(); - vec![RemoteExpr::ColumnRef { - span: None, - id: group_by_key_index, - data_type: group_by_key_data_type, - display_name: "_group_by_key".to_string(), - }] }; PhysicalPlan::Exchange(Exchange { diff --git a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs index b47f9f00d031..805d83af9bbb 100644 --- a/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs +++ b/src/query/sql/src/executor/physical_plans/physical_aggregate_partial.rs @@ -46,57 +46,28 @@ impl AggregatePartial { pub fn output_schema(&self) -> Result { let input_schema = self.input.output_schema()?; - if self.enable_experimental_aggregate_hashtable { - let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.len()); - for agg in self.agg_funcs.iter() { - fields.push(DataField::new( - &agg.output_column.to_string(), - DataType::Binary, - )); - } - - let group_types = self - .group_by - .iter() - .map(|index| { - Ok(input_schema - .field_with_name(&index.to_string())? - .data_type() - .clone()) - }) - .collect::>>()?; - - for (idx, data_type) in self.group_by.iter().zip(group_types.iter()) { - fields.push(DataField::new(&idx.to_string(), data_type.clone())); - } - return Ok(DataSchemaRefExt::create(fields)); - } - - let mut fields = - Vec::with_capacity(self.agg_funcs.len() + self.group_by.is_empty() as usize); + let mut fields = Vec::with_capacity(self.agg_funcs.len() + self.group_by.len()); for agg in self.agg_funcs.iter() { fields.push(DataField::new( &agg.output_column.to_string(), DataType::Binary, )); } - if !self.group_by.is_empty() { - let method = DataBlock::choose_hash_method_with_types( - &self - .group_by - .iter() - .map(|index| { - Ok(input_schema - .field_with_name(&index.to_string())? - .data_type() - .clone()) - }) - .collect::>>()?, - false, - )?; - fields.push(DataField::new("_group_by_key", method.data_type())); - } + let group_types = self + .group_by + .iter() + .map(|index| { + Ok(input_schema + .field_with_name(&index.to_string())? + .data_type() + .clone()) + }) + .collect::>>()?; + + for (idx, data_type) in self.group_by.iter().zip(group_types.iter()) { + fields.push(DataField::new(&idx.to_string(), data_type.clone())); + } Ok(DataSchemaRefExt::create(fields)) } } diff --git a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs index ba0253ea4754..053cdb8e56e0 100644 --- a/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs +++ b/src/query/storages/fuse/src/operations/read/runtime_filter_prunner.rs @@ -107,7 +107,7 @@ pub(crate) fn update_bitmap_with_bloom_filter( ) -> Result<()> { let data_type = column.data_type(); let num_rows = column.len(); - let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()], false)?; + let method = DataBlock::choose_hash_method_with_types(&[data_type.clone()])?; let columns = &[column]; let group_columns = columns.into(); let mut idx = 0; @@ -125,19 +125,6 @@ pub(crate) fn update_bitmap_with_bloom_filter( _ => unreachable!(), } } - HashMethodKind::DictionarySerializer(method) => { - let key_state = method.build_keys_state(group_columns, num_rows)?; - match key_state { - KeysState::Dictionary { dictionaries, .. } => dictionaries.iter().for_each(|key| { - let hash = key.fast_hash(); - if filter.contains(&hash) { - bitmap.set(idx, true); - } - idx += 1; - }), - _ => unreachable!(), - } - } HashMethodKind::SingleBinary(method) => { let key_state = method.build_keys_state(group_columns, num_rows)?; match key_state {